Repository: yahoo/HaloDB
Branch: master
Commit: 767d2357a4c9
Files: 117
Total size: 575.4 KB
Directory structure:
gitextract_fajmuc1o/
├── .github/
│ └── workflows/
│ ├── maven-publish.yml
│ └── maven.yml
├── .gitignore
├── .travis.yml
├── CHANGELOG.md
├── CONTRIBUTING.md
├── CONTRIBUTORS.md
├── Code-of-Conduct.md
├── LICENSE
├── NOTICE
├── README.md
├── benchmarks/
│ ├── README.md
│ ├── pom.xml
│ └── src/
│ └── main/
│ └── java/
│ └── com/
│ └── oath/
│ └── halodb/
│ └── benchmarks/
│ ├── BenchmarkTool.java
│ ├── Benchmarks.java
│ ├── HaloDBStorageEngine.java
│ ├── KyotoStorageEngine.java
│ ├── RandomDataGenerator.java
│ ├── RocksDBStorageEngine.java
│ └── StorageEngine.java
├── docs/
│ ├── WhyHaloDB.md
│ └── benchmarks.md
├── pom.xml
└── src/
├── main/
│ └── java/
│ └── com/
│ └── oath/
│ └── halodb/
│ ├── CompactionManager.java
│ ├── Constants.java
│ ├── DBDirectory.java
│ ├── DBMetaData.java
│ ├── FileUtils.java
│ ├── HaloDB.java
│ ├── HaloDBException.java
│ ├── HaloDBFile.java
│ ├── HaloDBInternal.java
│ ├── HaloDBIterator.java
│ ├── HaloDBKeyIterator.java
│ ├── HaloDBOptions.java
│ ├── HaloDBStats.java
│ ├── HashAlgorithm.java
│ ├── HashTableUtil.java
│ ├── HashTableValueSerializer.java
│ ├── Hasher.java
│ ├── InMemoryIndex.java
│ ├── InMemoryIndexMetaData.java
│ ├── InMemoryIndexMetaDataSerializer.java
│ ├── IndexFile.java
│ ├── IndexFileEntry.java
│ ├── JNANativeAllocator.java
│ ├── KeyBuffer.java
│ ├── LongArrayList.java
│ ├── MemoryPoolAddress.java
│ ├── MemoryPoolChunk.java
│ ├── MemoryPoolHashEntries.java
│ ├── NativeMemoryAllocator.java
│ ├── NonMemoryPoolHashEntries.java
│ ├── OffHeapHashTable.java
│ ├── OffHeapHashTableBuilder.java
│ ├── OffHeapHashTableImpl.java
│ ├── OffHeapHashTableStats.java
│ ├── Record.java
│ ├── RecordKey.java
│ ├── Segment.java
│ ├── SegmentNonMemoryPool.java
│ ├── SegmentStats.java
│ ├── SegmentWithMemoryPool.java
│ ├── TombstoneEntry.java
│ ├── TombstoneFile.java
│ ├── Uns.java
│ ├── UnsExt.java
│ ├── UnsExt8.java
│ ├── UnsafeAllocator.java
│ ├── Utils.java
│ ├── Versions.java
│ └── histo/
│ └── EstimatedHistogram.java
└── test/
├── java/
│ └── com/
│ └── oath/
│ └── halodb/
│ ├── CheckOffHeapHashTable.java
│ ├── CheckSegment.java
│ ├── CompactionWithErrorsTest.java
│ ├── CrossCheckTest.java
│ ├── DBDirectoryTest.java
│ ├── DBMetaDataTest.java
│ ├── DBRepairTest.java
│ ├── DataConsistencyDB.java
│ ├── DataConsistencyTest.java
│ ├── DoubleCheckOffHeapHashTableImpl.java
│ ├── FileUtilsTest.java
│ ├── HaloDBCompactionTest.java
│ ├── HaloDBDeletionTest.java
│ ├── HaloDBFileCompactionTest.java
│ ├── HaloDBFileTest.java
│ ├── HaloDBIteratorTest.java
│ ├── HaloDBKeyIteratorTest.java
│ ├── HaloDBOptionsTest.java
│ ├── HaloDBStatsTest.java
│ ├── HaloDBTest.java
│ ├── HashTableTestUtils.java
│ ├── HashTableUtilTest.java
│ ├── HashTableValueSerializerTest.java
│ ├── HasherTest.java
│ ├── IndexFileEntryTest.java
│ ├── KeyBufferTest.java
│ ├── LinkedImplTest.java
│ ├── LongArrayListTest.java
│ ├── MemoryPoolChunkTest.java
│ ├── NonMemoryPoolHashEntriesTest.java
│ ├── OffHeapHashTableBuilderTest.java
│ ├── RandomDataGenerator.java
│ ├── RecordTest.java
│ ├── RehashTest.java
│ ├── SegmentWithMemoryPoolTest.java
│ ├── SequenceNumberTest.java
│ ├── SyncWriteTest.java
│ ├── TestBase.java
│ ├── TestListener.java
│ ├── TestUtils.java
│ ├── TombstoneFileCleanUpTest.java
│ ├── TombstoneFileTest.java
│ ├── UnsTest.java
│ └── histo/
│ └── EstimatedHistogramTest.java
└── resources/
└── log4j2-test.xml
================================================
FILE CONTENTS
================================================
================================================
FILE: .github/workflows/maven-publish.yml
================================================
# This workflow will build a package using Maven and then publish it to GitHub packages when a release is created
# For more information see: https://github.com/actions/setup-java/blob/main/docs/advanced-usage.md#apache-maven-with-a-settings-path
name: Maven Package
on:
release:
types: [created]
jobs:
build:
runs-on: ubuntu-latest
permissions:
contents: read
packages: write
steps:
- uses: actions/checkout@v2
- name: Set up JDK 8
uses: actions/setup-java@v2
with:
java-version: '8'
distribution: 'adopt'
server-id: github # Value of the distributionManagement/repository/id field of the pom.xml
settings-path: ${{ github.workspace }} # location for the settings.xml file
- name: Publish to GitHub Packages Apache Maven
run: mvn deploy -s $GITHUB_WORKSPACE/settings.xml
env:
GITHUB_TOKEN: ${{ github.token }}
================================================
FILE: .github/workflows/maven.yml
================================================
# This workflow will build a Java project with Maven
# For more information see: https://help.github.com/actions/language-and-framework-guides/building-and-testing-java-with-maven
name: Java CI with Maven
on:
push:
branches: [ master ]
pull_request:
branches: [ master ]
jobs:
build:
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v2
- name: Set up JDK 8
uses: actions/setup-java@v2
with:
java-version: '8'
distribution: 'adopt'
- name: Build with Maven
run: mvn -B package --file pom.xml
================================================
FILE: .gitignore
================================================
target
.idea
halodb.iml
tmp/
================================================
FILE: .travis.yml
================================================
language: java
dist: trusty
jdk:
- oraclejdk8
================================================
FILE: CHANGELOG.md
================================================
# HaloDB Change Log
## 0.4.3 (08/20/2018)
* Sequence number, instead of relying on system time, is now a number incremented for each write operation.
* Include compaction rate in stats.
## 0.4.2 (08/06/2018)
* Handle the case where db crashes while it is being repaired due to error from a previous crash.
* _put_ operation in _HaloDB_ now returns a boolean value indicating the status of the operation.
## 0.4.1 (7/16/2018)
* Include version, checksum and max file size in META file.
* _maxFileSize_ in _HaloDBOptions_ now accepts only int values.
## 0.4.0 (7/11/2018)
* Implemented memory pool for in-memory index.
================================================
FILE: CONTRIBUTING.md
================================================
# How to contribute
First, thanks for taking the time to contribute to our project! The following information provides a guide for making contributions.
## Code of Conduct
By participating in this project, you agree to abide by the [Oath Code of Conduct](Code-of-Conduct.md). Everyone is welcome to submit a pull request or open an issue to improve the documentation, add improvements, or report bugs.
## How to Ask a Question
If you simply have a question that needs an answer, [create an issue](https://help.github.com/articles/creating-an-issue/), and label it as a question.
## How To Contribute
### Report a Bug or Request a Feature
If you encounter any bugs while using this software, or want to request a new feature or enhancement, feel free to [create an issue](https://help.github.com/articles/creating-an-issue/) to report it, make sure you add a label to indicate what type of issue it is.
### Contribute Code
Pull requests are welcome for bug fixes. If you want to implement something new, please [request a feature first](#report-a-bug-or-request-a-feature) so we can discuss it.
#### Creating a Pull Request
Please follow [best practices](https://github.com/trein/dev-best-practices/wiki/Git-Commit-Best-Practices) for creating git commits.
When your code is ready to be submitted, you can [submit a pull request](https://help.github.com/articles/creating-a-pull-request/) to begin the code review process.
================================================
FILE: CONTRIBUTORS.md
================================================
HaloDB was designed and implemented by [Arjun Mannaly](https://github.com/amannaly)
================================================
FILE: Code-of-Conduct.md
================================================
# Oath Open Source Code of Conduct
## Summary
This Code of Conduct is our way to encourage good behavior and discourage bad behavior in our open source community. We invite participation from many people to bring different perspectives to support this project. We pledge to do our part to foster a welcoming and professional environment free of harassment. We expect participants to communicate professionally and thoughtfully during their involvement with this project.
Participants may lose their good standing by engaging in misconduct. For example: insulting, threatening, or conveying unwelcome sexual content. We ask participants who observe conduct issues to report the incident directly to the project's Response Team at opensource-conduct@oath.com. Oath will assign a respondent to address the issue. We may remove harassers from this project.
This code does not replace the terms of service or acceptable use policies of the websites used to support this project. We acknowledge that participants may be subject to additional conduct terms based on their employment which may govern their online expressions.
## Details
This Code of Conduct makes our expectations of participants in this community explicit.
* We forbid harassment and abusive speech within this community.
* We request participants to report misconduct to the project’s Response Team.
* We urge participants to refrain from using discussion forums to play out a fight.
### Expected Behaviors
We expect participants in this community to conduct themselves professionally. Since our primary mode of communication is text on an online forum (e.g. issues, pull requests, comments, emails, or chats) devoid of vocal tone, gestures, or other context that is often vital to understanding, it is important that participants are attentive to their interaction style.
* **Assume positive intent.** We ask community members to assume positive intent on the part of other people’s communications. We may disagree on details, but we expect all suggestions to be supportive of the community goals.
* **Respect participants.** We expect participants will occasionally disagree. Even if we reject an idea, we welcome everyone’s participation. Open Source projects are learning experiences. Ask, explore, challenge, and then respectfully assert if you agree or disagree. If your idea is rejected, be more persuasive not bitter.
* **Welcoming to new members.** New members bring new perspectives. Some may raise questions that have been addressed before. Kindly point them to existing discussions. Everyone is new to every project once.
* **Be kind to beginners.** Beginners use open source projects to get experience. They might not be talented coders yet, and projects should not accept poor quality code. But we were all beginners once, and we need to engage kindly.
* **Consider your impact on others.** Your work will be used by others, and you depend on the work of others. We expect community members to be considerate and establish a balance their self-interest with communal interest.
* **Use words carefully.** We may not understand intent when you say something ironic. Poe’s Law suggests that without an emoticon people will misinterpret sarcasm. We ask community members to communicate plainly.
* **Leave with class.** When you wish to resign from participating in this project for any reason, you are free to fork the code and create a competitive project. Open Source explicitly allows this. Your exit should not be dramatic or bitter.
### Unacceptable Behaviors
Participants remain in good standing when they do not engage in misconduct or harassment. To elaborate:
* **Don't be a bigot.** Calling out project members by their identity or background in a negative or insulting manner. This includes, but is not limited to, slurs or insinuations related to protected or suspect classes e.g. race, color, citizenship, national origin, political belief, religion, sexual orientation, gender identity and expression, age, size, culture, ethnicity, genetic features, language, profession, national minority statue, mental or physical ability.
* **Don't insult.** Insulting remarks about a person’s lifestyle practices.
* **Don't dox.** Revealing private information about other participants without explicit permission.
* **Don't intimidate.** Threats of violence or intimidation of any project member.
* **Don't creep.** Unwanted sexual attention or content unsuited for the subject of this project.
* **Don't disrupt.** Sustained disruptions in a discussion.
* **Let us help.** Refusal to assist the Response Team to resolve an issue in the community.
We do not list all forms of harassment, nor imply some forms of harassment are not worthy of action. Any participant who *feels* harassed or *observes* harassment, should report the incident. Victim of harassment should not address grievances in the public forum, as this often intensifies the problem. Report it, and let us address it off-line.
### Reporting Issues
If you experience or witness misconduct, or have any other concerns about the conduct of members of this project, please report it by contacting our Response Team at opensource-conduct@oath.com who will handle your report with discretion. Your report should include:
* Your preferred contact information. We cannot process anonymous reports.
* Names (real or usernames) of those involved in the incident.
* Your account of what occurred, and if the incident is ongoing. Please provide links to or transcripts of the publicly available records (e.g. a mailing list archive or a public IRC logger), so that we can review it.
* Any additional information that may be helpful to achieve resolution.
After filing a report, a representative will contact you directly to review the incident and ask additional questions. If a member of the Oath Response Team is named in an incident report, that member will be recused from handling your incident. If the complaint originates from a member of the Response Team, it will be addressed by a different member of the Response Team. We will consider reports to be confidential for the purpose of protecting victims of abuse.
### Scope
Oath will assign a Response Team member with admin rights on the project and legal rights on the project copyright. The Response Team is empowered to restrict some privileges to the project as needed. Since this project is governed by an open source license, any participant may fork the code under the terms of the project license. The Response Team’s goal is to preserve the project if possible, and will restrict or remove participation from those who disrupt the project.
This code does not replace the terms of service or acceptable use policies that are provided by the websites used to support this community. Nor does this code apply to communications or actions that take place outside of the context of this community. Many participants in this project are also subject to codes of conduct based on their employment. This code is a social-contract that informs participants of our social expectations. It is not a terms of service or legal contract.
## License and Acknowledgment.
This text is shared under the [CC-BY-4.0 license](https://creativecommons.org/licenses/by/4.0/). This code is based on a study conducted by the [TODO Group](https://todogroup.org/) of many codes used in the open source community. If you have feedback about this code, contact our Response Team at the address listed above.
================================================
FILE: LICENSE
================================================
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction, and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all other entities that control, are controlled by, or are under common control with that entity. For the purposes of this definition, "control" means (i) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (ii) ownership of fifty percent (50%) or more of the outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications, including but not limited to software source code, documentation source, and configuration files.
"Object" form shall mean any form resulting from mechanical transformation or translation of a Source form, including but not limited to compiled object code, generated documentation, and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or Object form, made available under the License, as indicated by a copyright notice that is included in or attached to the work (an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object form, that is based on (or derived from) the Work and for which the editorial revisions, annotations, elaborations, or other modifications represent, as a whole, an original work of authorship. For the purposes of this License, Derivative Works shall not include works that remain separable from, or merely link (or bind by name) to the interfaces of, the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including the original version of the Work and any modifications or additions to that Work or Derivative Works thereof, that is intentionally submitted to Licensor for inclusion in the Work by the copyright owner or by an individual or Legal Entity authorized to submit on behalf of the copyright owner. For the purposes of this definition, "submitted" means any form of electronic, verbal, or written communication sent to the Licensor or its representatives, including but not limited to communication on electronic mailing lists, source code control systems, and issue tracking systems that are managed by, or on behalf of, the Licensor for the purpose of discussing and improving the Work, but excluding communication that is conspicuously marked or otherwise designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity on behalf of whom a Contribution has been received by Licensor and subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable copyright license to reproduce, prepare Derivative Works of, publicly display, publicly perform, sublicense, and distribute the Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable (except as stated in this section) patent license to make, have made, use, offer to sell, sell, import, and otherwise transfer the Work, where such license applies only to those patent claims licensable by such Contributor that are necessarily infringed by their Contribution(s) alone or by combination of their Contribution(s) with the Work to which such Contribution(s) was submitted. If You institute patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Work or a Contribution incorporated within the Work constitutes direct or contributory patent infringement, then any patent licenses granted to You under this License for that Work shall terminate as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the Work or Derivative Works thereof in any medium, with or without modifications, and in Source or Object form, provided that You meet the following conditions:
You must give any other recipients of the Work or Derivative Works a copy of this License; and
You must cause any modified files to carry prominent notices stating that You changed the files; and
You must retain, in the Source form of any Derivative Works that You distribute, all copyright, patent, trademark, and attribution notices from the Source form of the Work, excluding those notices that do not pertain to any part of the Derivative Works; and
If the Work includes a "NOTICE" text file as part of its distribution, then any Derivative Works that You distribute must include a readable copy of the attribution notices contained within such NOTICE file, excluding those notices that do not pertain to any part of the Derivative Works, in at least one of the following places: within a NOTICE text file distributed as part of the Derivative Works; within the Source form or documentation, if provided along with the Derivative Works; or, within a display generated by the Derivative Works, if and wherever such third-party notices normally appear. The contents of the NOTICE file are for informational purposes only and do not modify the License. You may add Your own attribution notices within Derivative Works that You distribute, alongside or as an addendum to the NOTICE text from the Work, provided that such additional attribution notices cannot be construed as modifying the License.
You may add Your own copyright statement to Your modifications and may provide additional or different license terms and conditions for use, reproduction, or distribution of Your modifications, or for any such Derivative Works as a whole, provided Your use, reproduction, and distribution of the Work otherwise complies with the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise, any Contribution intentionally submitted for inclusion in the Work by You to the Licensor shall be under the terms and conditions of this License, without any additional terms or conditions. Notwithstanding the above, nothing herein shall supersede or modify the terms of any separate license agreement you may have executed with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade names, trademarks, service marks, or product names of the Licensor, except as required for reasonable and customary use in describing the origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or agreed to in writing, Licensor provides the Work (and each Contributor provides its Contributions) on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied, including, without limitation, any warranties or conditions of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A PARTICULAR PURPOSE. You are solely responsible for determining the appropriateness of using or redistributing the Work and assume any risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory, whether in tort (including negligence), contract, or otherwise, unless required by applicable law (such as deliberate and grossly negligent acts) or agreed to in writing, shall any Contributor be liable to You for damages, including any direct, indirect, special, incidental, or consequential damages of any character arising as a result of this License or out of the use or inability to use the Work (including but not limited to damages for loss of goodwill, work stoppage, computer failure or malfunction, or any and all other commercial damages or losses), even if such Contributor has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing the Work or Derivative Works thereof, You may choose to offer, and charge a fee for, acceptance of support, warranty, indemnity, or other liability obligations and/or rights consistent with this License. However, in accepting such obligations, You may act only on Your own behalf and on Your sole responsibility, not on behalf of any other Contributor, and only if You agree to indemnify, defend, and hold each Contributor harmless for any liability incurred by, or claims asserted against, such Contributor by reason of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
================================================
FILE: NOTICE
================================================
=========================================================================
NOTICE file for use with, and corresponding to Section 4 of,
the Apache License, Version 2.0 (http://www.apache.org/licenses/LICENSE-2.0)
in this case for the HaloDB project
=========================================================================
This project contains software developed by Robert Stupp.
OHC (https://github.com/snazy/ohc)
Java Off-Heap-Cache, licensed under APLv2
Copyright (C) 2014 Robert Stupp, Koeln, Germany, robert-stupp.de
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
================================================
FILE: README.md
================================================
# HaloDB
[](https://travis-ci.org/yahoo/HaloDB)
[ ](https://bintray.com/yahoo/maven/halodb/_latestVersion)
HaloDB is a fast and simple embedded key-value store written in Java. HaloDB is suitable for IO bound workloads, and is capable of handling high throughput reads and writes at submillisecond latencies.
HaloDB was written for a high-throughput, low latency distributed key-value database that powers multiple ad platforms at Yahoo, therefore all its design choices and optimizations were
primarily for this use case.
Basic design principles employed in HaloDB are not new. Refer to this [document](docs/WhyHaloDB.md) for more details about the motivation for HaloDB and its inspirations.
HaloDB comprises of two main components: an index in memory which stores all the keys, and append-only log files on
the persistent layer which stores all the data. To reduce Java garbage collection pressure the index
is allocated in native memory, outside the Java heap.

### Basic Operations.
```java
// Open a db with default options.
HaloDBOptions options = new HaloDBOptions();
// Size of each data file will be 1GB.
options.setMaxFileSize(1024 * 1024 * 1024);
// Size of each tombstone file will be 64MB
// Large file size mean less file count but will slow down db open time. But if set
// file size too small, it will result large amount of tombstone files under db folder
options.setMaxTombstoneFileSize(64 * 1024 * 1024);
// Set the number of threads used to scan index and tombstone files in parallel
// to build in-memory index during db open. It must be a positive number which is
// not greater than Runtime.getRuntime().availableProcessors().
// It is used to speed up db open time.
options.setBuildIndexThreads(8);
// The threshold at which page cache is synced to disk.
// data will be durable only if it is flushed to disk, therefore
// more data will be lost if this value is set too high. Setting
// this value too low might interfere with read and write performance.
options.setFlushDataSizeBytes(10 * 1024 * 1024);
// The percentage of stale data in a data file at which the file will be compacted.
// This value helps control write and space amplification. Increasing this value will
// reduce write amplification but will increase space amplification.
// This along with the compactionJobRate below is the most important setting
// for tuning HaloDB performance. If this is set to x then write amplification
// will be approximately 1/x.
options.setCompactionThresholdPerFile(0.7);
// Controls how fast the compaction job should run.
// This is the amount of data which will be copied by the compaction thread per second.
// Optimal value depends on the compactionThresholdPerFile option.
options.setCompactionJobRate(50 * 1024 * 1024);
// Setting this value is important as it helps to preallocate enough
// memory for the off-heap cache. If the value is too low the db might
// need to rehash the cache. For a db of size n set this value to 2*n.
options.setNumberOfRecords(100_000_000);
// Delete operation for a key will write a tombstone record to a tombstone file.
// the tombstone record can be removed only when all previous version of that key
// has been deleted by the compaction job.
// enabling this option will delete during startup all tombstone records whose previous
// versions were removed from the data file.
options.setCleanUpTombstonesDuringOpen(true);
// HaloDB does native memory allocation for the in-memory index.
// Enabling this option will release all allocated memory back to the kernel when the db is closed.
// This option is not necessary if the JVM is shutdown when the db is closed, as in that case
// allocated memory is released automatically by the kernel.
// If using in-memory index without memory pool this option,
// depending on the number of records in the database,
// could be a slow as we need to call _free_ for each record.
options.setCleanUpInMemoryIndexOnClose(false);
// ** settings for memory pool **
options.setUseMemoryPool(true);
// Hash table implementation in HaloDB is similar to that of ConcurrentHashMap in Java 7.
// Hash table is divided into segments and each segment manages its own native memory.
// The number of segments is twice the number of cores in the machine.
// A segment's memory is further divided into chunks whose size can be configured here.
options.setMemoryPoolChunkSize(2 * 1024 * 1024);
// using a memory pool requires us to declare the size of keys in advance.
// Any write request with key length greater than the declared value will fail, but it
// is still possible to store keys smaller than this declared size.
options.setFixedKeySize(8);
// Represents a database instance and provides all methods for operating on the database.
HaloDB db = null;
// The directory will be created if it doesn't exist and all database files will be stored in this directory
String directory = "directory";
// Open the database. Directory will be created if it doesn't exist.
// If we are opening an existing database HaloDB needs to scan all the
// index files to create the in-memory index, which, depending on the db size, might take a few minutes.
db = HaloDB.open(directory, options);
// key and values are byte arrays. Key size is restricted to 128 bytes.
byte[] key1 = Ints.toByteArray(200);
byte[] value1 = "Value for key 1".getBytes();
byte[] key2 = Ints.toByteArray(300);
byte[] value2 = "Value for key 2".getBytes();
// add the key-value pair to the database.
db.put(key1, value1);
db.put(key2, value2);
// read the value from the database.
value1 = db.get(key1);
value2 = db.get(key2);
// delete a key from the database.
db.delete(key1);
// Open an iterator and iterate through all the key-value records.
HaloDBIterator iterator = db.newIterator();
while (iterator.hasNext()) {
Record record = iterator.next();
System.out.println(Ints.fromByteArray(record.getKey()));
System.out.println(new String(record.getValue()));
}
// get stats and print it.
HaloDBStats stats = db.stats();
System.out.println(stats.toString());
// reset stats
db.resetStats();
// pause background compaction thread.
// if a file is being compacted the thread
// will block until the compaction is complete.
db.pauseCompaction();
// resume background compaction thread.
db.resumeCompaction();
// repeatedly calling pause/resume compaction methods will have no effect.
// Close the database.
db.close();
```
Binaries for HaloDB are hosted on [Bintray](https://bintray.com/yahoo).
``` xml
com.oath.halodbhalodbx.y.xyahoo-bintrayyahoo-bintrayhttps://yahoo.bintray.com/maven
```
### Read, Write and Space amplification.
Read amplification in HaloDB is always 1—for a read request it needs to do at most one disk lookup—hence it is well suited for
read latency critical workloads. HaloDB provides a configuration which can be tuned to control write amplification
and space amplification, both of which trade-off with each other; HaloDB has a background compaction thread which removes stale data
from the DB. The percentage of stale data at which a file is compacted can be controlled. Increasing this value will increase space amplification
but will reduce write amplification. For example if the value is set to 50% then write amplification will be approximately 2
### Durability and Crash recovery.
Write Ahead Logs (WAL) are usually used by databases for crash recovery. Since for HaloDB WAL _is the_ database crash recovery
is easier and faster.
HaloDB does not flush writes to disk immediately, but, for performance reasons, writes only to the OS page cache. The cache is synced to
disk once a configurable size is reached. In the event of a power loss, the data not flushed to disk will be lost. This compromise
between performance and durability is a necessary one.
In the event of a power loss and data corruption, HaloDB will scan and discard corrupted records. Since the write thread and compaction
thread could be writing to at most two files at a time only those files need to be repaired and hence recovery times are very short.
In the event of a power loss HaloDB offers the following consistency guarantees:
* Writes are atomic.
* Inserts and updates are committed to disk in the same order they are received.
* When inserts/updates and deletes are interleaved total ordering is not guaranteed, but partial ordering is guaranteed for inserts/updates and deletes.
### In-memory index.
HaloDB stores all keys and their associated metadata in an index in memory. The size of this index, depending on the
number and length of keys, can be quite big. Therefore, storing this in the Java Heap is a non-starter for a
performance critical storage engine. HaloDB solves this problem by storing the index in native memory,
outside the heap. There are two variants of the index; one with a memory pool and the other
without it. Using the memory pool helps to reduce the memory footprint of the index and reduce
fragmentation, but requires fixed size keys. A billion 8 byte keys
currently takes around 44GB of memory with memory pool and around 64GB without memory pool.
The size of the keys when using a memory pool should be declared in advance, and although this imposes an
upper limit on the size of the keys it is still possible to store keys smaller than this declared size.
Without the memory pool, HaloDB needs to allocate native memory for every write request. Therefore,
memory fragmentation could be an issue. Using [jemalloc](http://jemalloc.net/) is highly recommended as it
provides a significant reduction in the cache's memory footprint and fragmentation.
### Delete operations.
Delete operation for a key will add a tombstone record to a tombstone file, which is distinct from the data files.
This design has the advantage that the tombstone record once written need not be copied again during compaction, but
the drawback is that in case of a power loss HaloDB cannot guarantee total ordering when put and delete operations are
interleaved (although partial ordering for both is guaranteed).
### DB open time
Open db could take a few minutes, depends on number of records and tombstones. If the db open time is critical to your
use case, please keep tombstone file size relatively small and increase the number of threads used in building index.
See the option setting section in example code above. As best practice, set tombstone file size at 64MB and set build
index threads to number of available processors divided by number of dbs being opened simultaneously.
### System requirements.
* HaloDB requires Java 8 to run, but has not yet been tested with newer Java versions.
* HaloDB has been tested on Linux running on x86 and on MacOS. It may run on other platforms, but this hasn't been verified yet.
* For performance disable Transparent Huge Pages and swapping (vm.swappiness=0).
* If a thread is interrupted JVM will close those file channels the thread was operating on.
Therefore, don't interrupt threads while they are doing IO operations.
### Restrictions.
* Size of keys is restricted to 128 bytes.
* HaloDB don't support range scans or ordered access.
# Benchmarks.
[Benchmarks](docs/benchmarks.md).
# Contributing
Contributions are most welcome. Please refer to the [CONTRIBUTING](https://github.com/yahoo/HaloDB/blob/master/CONTRIBUTING.md) guide
# Credits
HaloDB was written by [Arjun Mannaly](https://github.com/amannaly).
# License
HaloDB is released under the Apache License, Version 2.0
================================================
FILE: benchmarks/README.md
================================================
# Storage Engine Benchmark Tool.
Build the package using **mvn clean package** This will create a far jar *target/storage-engine-benchmark-1.0.jar*
Different benchmarks can be run using:
`java -jar storage-engine-benchmark-1.0-SNAPSHOT.jar `
Different benchmark types are defined [here](https://github.com/yahoo/HaloDB/blob/master/benchmarks/src/main/java/com/oath/halodb/benchmarks/Benchmarks.java).
================================================
FILE: benchmarks/pom.xml
================================================
4.0.0mannalystorage-engine-benchmark1.0jarstorage-engine-benchmarkhttp://maven.apache.orgUTF-8org.rocksdbrocksdbjni5.7.2com.oath.halodbhalodb0.4.2com.fallabskyotocabinet-java1.16com.google.guavaguava19.0org.hdrhistogramHdrHistogram2.1.9org.slf4jslf4j-simple1.8.0-alpha2org.apache.maven.pluginsmaven-compiler-plugin3.5.11.81.8org.apache.maven.pluginsmaven-shade-plugin2.3packageshadecom.oath.halodb.benchmarks.BenchmarkTool*:*META-INF/*.SFMETA-INF/*.DSAMETA-INF/*.RSAhalodb-bintrayhalodb-bintrayhttps://yahoo.bintray.com/maventruefalse
================================================
FILE: benchmarks/src/main/java/com/oath/halodb/benchmarks/BenchmarkTool.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb.benchmarks;
import com.google.common.primitives.Longs;
import com.google.common.util.concurrent.RateLimiter;
import org.HdrHistogram.Histogram;
import java.io.File;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.Arrays;
import java.util.Date;
import java.util.Random;
import java.util.concurrent.TimeUnit;
public class BenchmarkTool {
private static SimpleDateFormat sdf = new SimpleDateFormat("HH:mm:ss");
// adjust HaloDB number of records accordingly.
private final static int numberOfRecords = 500_000_000;
private static volatile boolean isReadComplete = false;
private static final int numberOfReads = 640_000_000;
private static final int numberOfReadThreads = 32;
private static final int noOfReadsPerThread = numberOfReads / numberOfReadThreads; // 400 million.
private static final int writeMBPerSecond = 20 * 1024 * 1024;
private static final RateLimiter writeRateLimiter = RateLimiter.create(writeMBPerSecond);
private static final int recordSize = 1024;
private static final int seed = 100;
private static final Random random = new Random(seed);
private static RandomDataGenerator randomDataGenerator = new RandomDataGenerator(seed);
public static void main(String[] args) throws Exception {
String directoryName = args[0];
String benchmarkType = args[1];
Benchmarks benchmark = null;
try {
benchmark = Benchmarks.valueOf(benchmarkType);
}
catch (IllegalArgumentException e) {
System.out.println("Benchmarks should be one of " + Arrays.toString(Benchmarks.values()));
System.exit(1);
}
System.out.println("Running benchmark " + benchmark);
File dir = new File(directoryName);
// select different storage engines here.
final StorageEngine db = new HaloDBStorageEngine(dir, numberOfRecords);
//final StorageEngine db = new RocksDBStorageEngine(dir, numberOfRecords);
//final StorageEngine db = new KyotoStorageEngine(dir, numberOfRecords);
db.open();
System.out.println("Opened the database.");
switch (benchmark) {
case FILL_SEQUENCE: createDB(db, true);break;
case FILL_RANDOM: createDB(db, false);break;
case READ_RANDOM: readRandom(db, numberOfReadThreads);break;
case RANDOM_UPDATE: update(db);break;
case READ_AND_UPDATE: updateWithReads(db);
}
db.close();
}
private static void createDB(StorageEngine db, boolean isSequential) {
long start = System.currentTimeMillis();
byte[] value;
long dataSize = 0;
for (int i = 0; i < numberOfRecords; i++) {
value = randomDataGenerator.getData(recordSize);
dataSize += (long)value.length;
byte[] key = isSequential ? longToBytes(i) : longToBytes(random.nextInt(numberOfRecords));
db.put(key, value);
if (i % 1_000_000 == 0) {
System.out.printf("%s: Wrote %d records\n", DateFormat.getTimeInstance().format(new Date()), i);
}
}
long end = System.currentTimeMillis();
long time = (end - start) / 1000;
System.out.println("Completed writing data in " + time);
System.out.printf("Write rate %d MB/sec\n", dataSize / time / 1024l / 1024l);
System.out.println("Size of database " + db.size());
}
private static void update(StorageEngine db) {
long start = System.currentTimeMillis();
byte[] value;
long dataSize = 0;
for (int i = 0; i < numberOfRecords; i++) {
value = randomDataGenerator.getData(recordSize);
writeRateLimiter.acquire(value.length);
dataSize += (long)value.length;
byte[] key = longToBytes(random.nextInt(numberOfRecords));
db.put(key, value);
if (i % 1_000_000 == 0) {
System.out.printf("%s: Wrote %d records\n", DateFormat.getTimeInstance().format(new Date()), i);
}
}
long end = System.currentTimeMillis();
long time = (end - start) / 1000;
System.out.println("Completed over writing data in " + time);
System.out.printf("Write rate %d MB/sec\n", dataSize / time / 1024l / 1024l);
System.out.println("Size of database " + db.size());
}
private static void readRandom(StorageEngine db, int threads) {
Read[] reads = new Read[numberOfReadThreads];
long start = System.currentTimeMillis();
for (int i = 0; i < reads.length; i++) {
reads[i] = new Read(db, i);
reads[i].start();
}
for (Read r : reads) {
try {
r.join();
} catch (InterruptedException e) {
e.printStackTrace();
}
}
long time = (System.currentTimeMillis() - start) / 1000;
System.out.printf("Completed %d reads with %d threads in %d seconds\n", numberOfReads, numberOfReadThreads, time);
System.out.println("Operations per second - " + numberOfReads/time);
Histogram latencyHistogram = new Histogram(TimeUnit.SECONDS.toNanos(10), 3);
for(Read r : reads) {
latencyHistogram.add(r.latencyHistogram);
}
System.out.printf("Max value - %d\n", latencyHistogram.getMaxValue());
System.out.printf("Average value - %f\n", latencyHistogram.getMean());
System.out.printf("95th percentile - %d\n", latencyHistogram.getValueAtPercentile(95.0));
System.out.printf("99th percentile - %d\n", latencyHistogram.getValueAtPercentile(99.0));
System.out.printf("99.9th percentile - %d\n", latencyHistogram.getValueAtPercentile(99.9));
System.out.printf("99.99th percentile - %d\n", latencyHistogram.getValueAtPercentile(99.99));
}
private static void updateWithReads(StorageEngine db) {
Read[] reads = new Read[numberOfReadThreads];
Thread update = new Thread(new Runnable() {
@Override
public void run() {
long start = System.currentTimeMillis();
byte[] value;
long dataSize = 0, count = 0;
while (!isReadComplete) {
value = randomDataGenerator.getData(recordSize);
writeRateLimiter.acquire(value.length);
dataSize += (long)value.length;
byte[] key = longToBytes(random.nextInt(numberOfRecords));
db.put(key, value);
if (count++ % 1_000_000 == 0) {
System.out.printf("%s: Wrote %d records\n", DateFormat.getTimeInstance().format(new Date()), count);
}
}
long end = System.currentTimeMillis();
long time = (end - start) / 1000;
System.out.println("Completed over writing data in " + time);
System.out.println("Write operations per second - " + count/time);
System.out.printf("Write rate %d MB/sec\n", dataSize / time / 1024l / 1024l);
System.out.println("Size of database " + db.size());
}
});
long start = System.currentTimeMillis();
for (int i = 0; i < reads.length; i++) {
reads[i] = new Read(db, i);
reads[i].start();
}
update.start();
for(Read r : reads) {
try {
r.join();
} catch (InterruptedException e) {
e.printStackTrace();
}
}
long time = (System.currentTimeMillis() - start) / 1000;
isReadComplete = true;
long maxTime = -1;
for (Read r : reads) {
maxTime = Math.max(maxTime, r.time);
}
maxTime = maxTime / 1000;
System.out.println("Maximum time taken by a read thread to complete - " + maxTime);
System.out.printf("Completed %d reads with %d threads in %d seconds\n", numberOfReads, numberOfReadThreads, time);
System.out.println("Read operations per second - " + numberOfReads/time);
Histogram latencyHistogram = new Histogram(TimeUnit.SECONDS.toNanos(10), 3);
for(Read r : reads) {
latencyHistogram.add(r.latencyHistogram);
}
System.out.printf("Max value - %d\n", latencyHistogram.getMaxValue());
System.out.printf("Average value - %f\n", latencyHistogram.getMean());
System.out.printf("95th percentile - %d\n", latencyHistogram.getValueAtPercentile(95.0));
System.out.printf("99th percentile - %d\n", latencyHistogram.getValueAtPercentile(99.0));
System.out.printf("99.9th percentile - %d\n", latencyHistogram.getValueAtPercentile(99.9));
System.out.printf("99.99th percentile - %d\n", latencyHistogram.getValueAtPercentile(99.99));
}
static class Read extends Thread {
final int id;
final Random rand;
final StorageEngine db;
long time;
Histogram latencyHistogram = new Histogram(TimeUnit.SECONDS.toNanos(10), 3);
Read(StorageEngine db, int id) {
this.db = db;
this.id = id;
rand = new Random(seed + id);
}
@Override
public void run() {
long sum = 0, count = 0;
long start = System.currentTimeMillis();
while (count < noOfReadsPerThread) {
long id = (long)rand.nextInt(numberOfRecords);
long s = System.nanoTime();
byte[] value = db.get(longToBytes(id));
latencyHistogram.recordValue(System.nanoTime()-s);
count++;
if (value == null) {
System.out.println("NO value for key " +id);
continue;
}
if (count % 1_000_000 == 0) {
System.out.printf(printDate() + "Read: %d Completed %d reads\n", this.id, count);
}
sum += value.length;
}
time = (System.currentTimeMillis() - start);
System.out.printf("Read: %d Completed in time %d\n", id, time);
}
}
public static byte[] longToBytes(long value) {
return Longs.toByteArray(value);
}
public static String printDate() {
return sdf.format(new Date()) + ": ";
}
}
================================================
FILE: benchmarks/src/main/java/com/oath/halodb/benchmarks/Benchmarks.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb.benchmarks;
public enum Benchmarks {
FILL_SEQUENCE,
FILL_RANDOM,
READ_RANDOM,
RANDOM_UPDATE,
READ_AND_UPDATE;
}
================================================
FILE: benchmarks/src/main/java/com/oath/halodb/benchmarks/HaloDBStorageEngine.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb.benchmarks;
import com.google.common.primitives.Ints;
import com.oath.halodb.HaloDB;
import com.oath.halodb.HaloDBException;
import com.oath.halodb.HaloDBOptions;
import java.io.File;
public class HaloDBStorageEngine implements StorageEngine {
private final File dbDirectory;
private HaloDB db;
private final long noOfRecords;
public HaloDBStorageEngine(File dbDirectory, long noOfRecords) {
this.dbDirectory = dbDirectory;
this.noOfRecords = noOfRecords;
}
@Override
public void put(byte[] key, byte[] value) {
try {
db.put(key, value);
} catch (HaloDBException e) {
e.printStackTrace();
}
}
@Override
public byte[] get(byte[] key) {
try {
return db.get(key);
} catch (HaloDBException e) {
e.printStackTrace();
}
return new byte[0];
}
@Override
public void delete(byte[] key) {
try {
db.delete(key);
} catch (HaloDBException e) {
e.printStackTrace();
}
}
@Override
public void open() {
HaloDBOptions opts = new HaloDBOptions();
opts.setMaxFileSize(1024*1024*1024);
opts.setCompactionThresholdPerFile(0.50);
opts.setFlushDataSizeBytes(10 * 1024 * 1024);
opts.setNumberOfRecords(Ints.checkedCast(2 * noOfRecords));
opts.setCompactionJobRate(135 * 1024 * 1024);
opts.setUseMemoryPool(true);
opts.setFixedKeySize(8);
try {
db = HaloDB.open(dbDirectory, opts);
} catch (HaloDBException e) {
e.printStackTrace();
}
}
@Override
public void close() {
if (db != null){
try {
db.close();
} catch (HaloDBException e) {
e.printStackTrace();
}
}
}
@Override
public long size() {
return db.size();
}
@Override
public void printStats() {
}
@Override
public String stats() {
return db.stats().toString();
}
}
================================================
FILE: benchmarks/src/main/java/com/oath/halodb/benchmarks/KyotoStorageEngine.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb.benchmarks;
import java.io.File;
import kyotocabinet.DB;
public class KyotoStorageEngine implements StorageEngine {
private final File dbDirectory;
private final int noOfRecords;
private final DB db = new DB(2);
public KyotoStorageEngine(File dbDirectory, int noOfRecords) {
this.dbDirectory = dbDirectory;
this.noOfRecords = noOfRecords;
}
@Override
public void open() {
int mode = DB.OWRITER | DB.OCREATE | DB.ONOREPAIR;
StringBuilder fileNameBuilder = new StringBuilder();
fileNameBuilder.append(dbDirectory.getPath()).append("/kyoto.kch");
// specifies the power of the alignment of record size
fileNameBuilder.append("#apow=").append(8);
// specifies the number of buckets of the hash table
fileNameBuilder.append("#bnum=").append(noOfRecords * 4);
// specifies the mapped memory size
fileNameBuilder.append("#msiz=").append(2_500_000_000l);
// specifies the unit step number of auto defragmentation
fileNameBuilder.append("#dfunit=").append(8);
System.out.printf("Creating %s\n", fileNameBuilder.toString());
if (!db.open(fileNameBuilder.toString(), mode)) {
throw new IllegalArgumentException(String.format("KC db %s open error: " + db.error(),
fileNameBuilder.toString()));
}
}
@Override
public void put(byte[] key, byte[] value) {
db.set(key, value);
}
@Override
public byte[] get(byte[] key) {
return db.get(key);
}
@Override
public void close() {
db.close();
}
@Override
public long size() {
return db.size();
}
}
================================================
FILE: benchmarks/src/main/java/com/oath/halodb/benchmarks/RandomDataGenerator.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb.benchmarks;
import java.util.Random;
public class RandomDataGenerator {
private final byte[] data;
private static final int size = 1003087;
private int position = 0;
public RandomDataGenerator(int seed) {
this.data = new byte[size];
Random random = new Random(seed);
random.nextBytes(data);
}
public byte[] getData(int length) {
byte[] b = new byte[length];
for (int i = 0; i < length; i++) {
if (position >= size) {
position = 0;
}
b[i] = data[position++];
}
return b;
}
}
================================================
FILE: benchmarks/src/main/java/com/oath/halodb/benchmarks/RocksDBStorageEngine.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb.benchmarks;
import org.rocksdb.CompressionType;
import org.rocksdb.Env;
import org.rocksdb.Options;
import org.rocksdb.RocksDB;
import org.rocksdb.RocksDBException;
import org.rocksdb.Statistics;
import org.rocksdb.WriteOptions;
import java.io.File;
import java.util.Arrays;
import java.util.List;
public class RocksDBStorageEngine implements StorageEngine {
private RocksDB db;
private Options options;
private Statistics statistics;
private WriteOptions writeOptions;
private final File dbDirectory;
public RocksDBStorageEngine(File dbDirectory, int noOfRecords) {
this.dbDirectory = dbDirectory;
}
@Override
public void put(byte[] key, byte[] value) {
try {
db.put(writeOptions, key, value);
} catch (RocksDBException e) {
e.printStackTrace();
}
}
@Override
public byte[] get(byte[] key) {
byte[] value = null;
try {
value = db.get(key);
} catch (RocksDBException e) {
e.printStackTrace();
}
return value;
}
@Override
public void open() {
options = new Options().setCreateIfMissing(true);
options.setStatsDumpPeriodSec(1000000);
options.setWriteBufferSize(128l * 1024 * 1024);
options.setMaxWriteBufferNumber(3);
options.setMaxBackgroundCompactions(20);
Env env = Env.getDefault();
env.setBackgroundThreads(20, Env.COMPACTION_POOL);
options.setEnv(env);
// max size of L1 10 MB.
options.setMaxBytesForLevelBase(10485760);
options.setTargetFileSizeBase(67108864);
options.setLevel0FileNumCompactionTrigger(4);
options.setLevel0SlowdownWritesTrigger(6);
options.setLevel0StopWritesTrigger(12);
options.setNumLevels(6);
options.setDeleteObsoleteFilesPeriodMicros(300000000);
options.setAllowMmapReads(false);
options.setCompressionType(CompressionType.SNAPPY_COMPRESSION);
System.out.printf("maxBackgroundCompactions %d \n", options.maxBackgroundCompactions());
System.out.printf("minWriteBufferNumberToMerge %d \n", options.minWriteBufferNumberToMerge());
System.out.printf("maxWriteBufferNumberToMaintain %d \n", options.maxWriteBufferNumberToMaintain());
System.out.printf("level0FileNumCompactionTrigger %d \n", options.level0FileNumCompactionTrigger());
System.out.printf("maxBytesForLevelBase %d \n", options.maxBytesForLevelBase());
System.out.printf("maxBytesForLevelMultiplier %f \n", options.maxBytesForLevelMultiplier());
System.out.printf("targetFileSizeBase %d \n", options.targetFileSizeBase());
System.out.printf("targetFileSizeMultiplier %d \n", options.targetFileSizeMultiplier());
List compressionLevels =
Arrays.asList(
CompressionType.NO_COMPRESSION,
CompressionType.NO_COMPRESSION,
CompressionType.SNAPPY_COMPRESSION,
CompressionType.SNAPPY_COMPRESSION,
CompressionType.SNAPPY_COMPRESSION,
CompressionType.SNAPPY_COMPRESSION
);
options.setCompressionPerLevel(compressionLevels);
System.out.printf("compressionPerLevel %s \n", options.compressionPerLevel());
System.out.printf("numLevels %s \n", options.numLevels());
writeOptions = new WriteOptions();
writeOptions.setDisableWAL(true);
System.out.printf("WAL is disabled - %s \n", writeOptions.disableWAL());
try {
db = RocksDB.open(options, dbDirectory.getPath());
} catch (RocksDBException e) {
e.printStackTrace();
}
}
@Override
public void close() {
//statistics.close();
options.close();
writeOptions.close();
db.close();
}
}
================================================
FILE: benchmarks/src/main/java/com/oath/halodb/benchmarks/StorageEngine.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb.benchmarks;
public interface StorageEngine {
void put(byte[] key, byte[] value);
default String stats() { return "";}
byte[] get(byte[] key);
default void delete(byte[] key) {};
void open();
void close();
default long size() {return 0;}
default void printStats() {
}
}
================================================
FILE: docs/WhyHaloDB.md
================================================
# HaloDB at Yahoo.
At Yahoo, we built this high throughput, low latency distributed key-value database that runs in multiple data centers in different parts for the world.
The database stores billions of records and handles millions of read and write requests per second with an SLA of 1 millisecond at the 99th percentile.
The data we have in this database must be persistent, and the working set is larger than what we can fit in memory.
Therefore, a key component of the database’s performance is a fast storage engine, for which we have relied on Kyoto Cabinet. Although Kyoto Cabinet has served us well,
it was designed primarily for a read-heavy workload and its write throughput started to be a bottleneck as we took on more write traffic.
There were also other issues we faced with Kyoto Cabinet; it takes up to an hour to repair a corrupted db, and takes hours to iterate over and update/delete records (which we have to do every night).
It also doesn't expose enough operational metrics or logs which makes resolving issues challenging. However, our primary concern was Kyoto Cabinet’s write performance,
which based on our projections, would have been a major obstacle for scaling the database; therefore, it was a good time to look for alternatives.
**These are the salient features of the database’s workload for which the storage engine will be used:**
* Small keys (8 bytes) and large values (10KB average)
* Both read and write throughput are high.
* Submillisecond read latency at the 99th percentile.
* Single writer thread.
* No need for ordered access or range scans.
* Working set is much larger than available memory, hence workload is IO bound.
* Database is written in Java.
## Why a new storage engine?
Although there are umpteen number of storage engines publicly available almost all use a variation of the following data structures to organize data on disk for fast lookup:
* __Hash table__: Kyoto Cabinet.
* __Log-structured merge tree__: LevelDB, RocksDB.
* __B-Tree/B+ Tree__: Berkeley DB, InnoDB.
Since our workload requires very high write throughput, Hash table and B-Tree based storage engines were not suitable as they need to do random writes.
Although modern SSDs have narrowed the gap between sequential and random write performance, sequential writes still have higher throughput, primarily due
to the reduced internal garbage collection load within the SSD. LSM trees also turned out to be unsuitable; benchmarking RocksDB on our workload showed
a write amplification of 10-12, therefore writing 100MB/sec to RocksDB meant that it will write more than 1 GB/sec to the SSD, clearly too high.
High write amplification of RocksDB is a property of the LSM data structure itself, thereby ruling out storage engines based on LSM trees.
LSM tree and B-Tree also maintain an ordering of keys to support efficient range scans, but the cost they pay is a read amplification greater than 1,
and for LSM tree, very high write amplification. Since our workload only does point lookups, we don’t want to pay the cost associated with storing data
in a format suitable for range scans.
These problems ruled out most of the publicly available and well maintained storage engines. Looking at alternate storage engine data structures led us to
explore ideas used in Log-structured storage systems. Here was a potential good fit; log-structured system only does sequential writes, an efficient
garbage collection implementation can keep write amplification low, and having an index in memory for the keys can give us a read amplification of one,
and we get transactional updates, snapshots, and quick crash recovery almost for free. Also in this scheme, there is no ordering of data and hence its
associated costs are not paid. We found that similar ideas have been used in [BitCask](https://github.com/basho/bitcask/blob/develop/doc/bitcask-intro.pdf)
and [Haystack](https://code.facebook.com/posts/685565858139515/needle-in-a-haystack-efficient-storage-of-billions-of-photos/).
But BitCask was written in Erlang, and since our database runs on the JVM running Erlang VM on the same box and talking to it from the JVM is something
that we didn’t want to do. Haystack, on the other hand, is a full-fledged distributed database optimized for storing photos, and its storage engine hasn’t been open sourced.
Therefore it was decided to write a new storage engine from scratch; thus the HaloDB project was initiated.
## Performance test results on our production workload.
The following chart shows the results of performance tests that we ran with production data against a performance test box with the same hardware as production boxes. The read requests were kept at 50,000 QPS while the write QPS was increased.

As you can see at the 99th percentile HaloDB read latency is an order of magnitude better than that of Kyoto Cabinet.
We recently upgraded our SSDs to PCIe NVMe SSDs. This has given us a significant performance boost and has narrowed the gap between HaloDB and Kyoto Cabinet,
but the difference is still significant:

Of course, these are results from performance tests, but nothing beats real data from hosts running in production.
Following chart shows the 99th percentile latency from a production server before and after migration to HaloDB.

HaloDB has thus given our production boxes a 50% improvement in capacity while consistently maintaining a sub-millisecond latency at the 99th percentile.
HaloDB also has fixed few other problems that we had with KyotoCabinet. The daily cleanup job that used to take upto 5 hours in Kyoto Cabinet is now complete in 90 minutes
with HaloDB due to its improved write throughput. Also, HaloDB takes only a few seconds to recover from a crash due to the fact that all log files,
once they are rolled over, are immutable. Hence, in the event of a crash only the last file that was being written to need to be repaired.
Whereas, with Kyoto Cabinet crash recovery used to take more than an hour to complete. And the metrics that HaloDB exposes gives us good insight into its internal state,
which was missing with Kyoto Cabinet.
================================================
FILE: docs/benchmarks.md
================================================
# Benchmarks
Benchmarks were run to compare HaloDB against RocksDB and KyotoCabinet.
KyotoCabinet was chosen as we were using it in production. RockDB was chosen as it is a well known storage engine
with good documentation and a large community. HaloDB and KyotoCabinet supports only a subset of RocksDB's features, therefore the comparison is not exactly fair to RocksDB.
All benchmarks were run on bare-metal box with the following specifications:
* 2 x Xeon E5-2680 2.50GHz (HT enabled, 24 cores, 48 threads)
* 128 GB of RAM.
* 1 Samsung PM863 960 GB SSD with XFS file system.
* RHEL 6 with kernel 2.6.32.
Key size was 8 bytes and value size 1024 bytes. Tests created a db with 500 million records with total size of approximately
500GB. Since this is significantly bigger than the available memory it will ensure that the workload will be IO bound, which is what HaloDB was primarily designed for.
Benchmark tool can be found [here](../benchmarks)
## Test 1: Fill Sequential.
Create a new db by inserting 500 million records in sorted key order.

DB size at the end of the test run.
| Storage Engine | GB |
| ------------- | --------- |
| HaloDB | 503 |
| KyotoCabinet | 609 |
| RocksDB | 487 |
## Test 2: Random Read
Measure random read performance with 32 threads doing _640 million reads_ in total. Read ahead was disabled for this test.

## Test 3: Random Update.
Perform 500 million updates to randomly selected records.

DB size at the end of the test run.
| Storage Engine | GB |
| ------------- | --------- |
| HaloDB | 556 |
| KyotoCabinet | 609 |
| RocksDB | 504 |
## Test 4: Fill Random.
Insert 500 million records into an empty db in random order.

## Test 5: Read and update.
32 threads doing a total of 640 million random reads and one thread doing random updates as fast as possible.

## Why HaloDB is fast.
HaloDB doesn't claim to be always better than RocksDB or KyotoCabinet. HaloDB was written for a specific type of workload, and therefore had
the advantage of optimizing for that workload; the trade-offs that HaloDB makes might make it sub-optimal for other workloads (best to run benchmarks to verify).
HaloDB also offers only a small subset of features compared to other storage engines like RocksDB.
All writes to HaloDB are sequential writes to append-only log files. HaloDB uses a background compaction job to clean up stale data.
The threshold at which a file is compacted can be tuned and this determines HaloDB's write amplification and space amplification.
A compaction threshold of 50% gives a write amplification of only 2, this coupled with the fact that we do only sequential writes
are the primary reasons for HaloDB’s high write throughput. Additionally, the only meta-data that HaloDB need to modify during writes are
those of the index in memory. The trade-off here is that HaloDB will occupy more space on disk.
To lookup the value for a key its corresponding metadata is first read from the in-memory index and then the value is read from disk.
Therefore each lookup request requires at most a single read from disk, giving us a read amplification of 1, and is primarily responsible
for HaloDB’s low read latencies. The trade-off here is that we need to store all the keys and their associated metadata in memory. HaloDB
also need to scan all the keys during startup to build the in-memory index. This, depending on the number of keys, might take a few minutes.
HaloDB avoids doing in-place updates and doesn't need record level locks. A type of MVCC is inherent in the design of all log-structured storage systems. This also helps with performance even under high read and write throughput.
HaloDB also doesn't support range scans and therefore doesn't pay the cost associated with storing data in a format suitable for efficient range scans.
================================================
FILE: pom.xml
================================================
4.0.0com.oath.halodbhalodb0.5.6jarHaloDBA fast, embedded, persistent key-value storage engine.http://maven.apache.orgArjun MannalyUTF-8org.slf4jslf4j-api1.7.12com.google.guavaguava18.0net.java.dev.jnajna4.1.0net.jpountz.lz4lz4true1.3org.hamcresthamcrest-all1.3testorg.apache.logging.log4jlog4j-core2.3testorg.apache.logging.log4jlog4j-slf4j-impl2.3testorg.testngtestng6.9.10testorg.jmockitjmockit1.38testorg.assertjassertj-core3.8.0testhttps://github.com/yahoo/HaloDBscm:git:git@github.com:yahoo/HaloDB.gitHEADorg.apache.maven.pluginsmaven-compiler-plugin3.5.11.81.8maven-surefire-plugin2.20.1listenercom.oath.halodb.TestListener-Xms2G -Xmx2Gmaven-release-plugin2.5.3v@{project.version}config*.propertiesgithubhttps://maven.pkg.github.com/yahoo/halodb
================================================
FILE: src/main/java/com/oath/halodb/CompactionManager.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.RateLimiter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.channels.FileChannel;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.locks.ReentrantLock;
class CompactionManager {
private static final Logger logger = LoggerFactory.getLogger(CompactionManager.class);
private final HaloDBInternal dbInternal;
private volatile boolean isRunning = false;
private final RateLimiter compactionRateLimiter;
private volatile HaloDBFile currentWriteFile = null;
private int currentWriteFileOffset = 0;
private final BlockingQueue compactionQueue;
private volatile CompactionThread compactionThread;
private volatile long numberOfRecordsCopied = 0;
private volatile long numberOfRecordsReplaced = 0;
private volatile long numberOfRecordsScanned = 0;
private volatile long sizeOfRecordsCopied = 0;
private volatile long sizeOfFilesDeleted = 0;
private volatile long totalSizeOfRecordsCopied = 0;
private volatile long compactionStartTime = System.currentTimeMillis();
private static final int STOP_SIGNAL = -10101;
private final ReentrantLock startStopLock = new ReentrantLock();
private volatile boolean stopInProgress = false;
CompactionManager(HaloDBInternal dbInternal) {
this.dbInternal = dbInternal;
this.compactionRateLimiter = RateLimiter.create(dbInternal.options.getCompactionJobRate());
this.compactionQueue = new LinkedBlockingQueue<>();
}
// If a file is being compacted we wait for it complete before stopping.
boolean stopCompactionThread(boolean closeCurrentWriteFile) throws IOException {
stopInProgress = true;
startStopLock.lock();
try {
isRunning = false;
if (isCompactionRunning()) {
// We don't want to call interrupt on compaction thread as it
// may interrupt IO operations and leave files in an inconsistent state.
// instead we use -10101 as a stop signal.
compactionQueue.put(STOP_SIGNAL);
compactionThread.join();
if (closeCurrentWriteFile && currentWriteFile != null) {
currentWriteFile.flushToDisk();
currentWriteFile.getIndexFile().flushToDisk();
currentWriteFile.close();
}
}
}
catch (InterruptedException e) {
logger.error("Error while waiting for compaction thread to stop", e);
return false;
}
finally {
stopInProgress = false;
startStopLock.unlock();
}
return true;
}
void startCompactionThread() {
startStopLock.lock();
try {
if (!isCompactionRunning()) {
isRunning = true;
compactionThread = new CompactionThread();
compactionThread.start();
}
} finally {
startStopLock.unlock();
}
}
void pauseCompactionThread() throws IOException {
logger.info("Pausing compaction thread ...");
stopCompactionThread(false);
}
void resumeCompaction() {
logger.info("Resuming compaction thread");
startCompactionThread();
}
int getCurrentWriteFileId() {
return currentWriteFile != null ? currentWriteFile.getFileId() : -1;
}
boolean submitFileForCompaction(int fileId) {
return compactionQueue.offer(fileId);
}
int noOfFilesPendingCompaction() {
return compactionQueue.size();
}
long getNumberOfRecordsCopied() {
return numberOfRecordsCopied;
}
long getNumberOfRecordsReplaced() {
return numberOfRecordsReplaced;
}
long getNumberOfRecordsScanned() {
return numberOfRecordsScanned;
}
long getSizeOfRecordsCopied() {
return sizeOfRecordsCopied;
}
long getSizeOfFilesDeleted() {
return sizeOfFilesDeleted;
}
long getCompactionJobRateSinceBeginning() {
long timeInSeconds = (System.currentTimeMillis() - compactionStartTime)/1000;
long rate = 0;
if (timeInSeconds > 0) {
rate = totalSizeOfRecordsCopied / timeInSeconds;
}
return rate;
}
void resetStats() {
numberOfRecordsCopied = numberOfRecordsReplaced
= numberOfRecordsScanned = sizeOfRecordsCopied = sizeOfFilesDeleted = 0;
}
boolean isCompactionRunning() {
return compactionThread != null && compactionThread.isAlive();
}
private class CompactionThread extends Thread {
private long unFlushedData = 0;
CompactionThread() {
super("CompactionThread");
setUncaughtExceptionHandler((t, e) -> {
logger.error("Compaction thread crashed", e);
if (currentWriteFile != null) {
try {
currentWriteFile.flushToDisk();
} catch (IOException ex) {
logger.error("Error while flushing " + currentWriteFile.getFileId() + " to disk", ex);
}
currentWriteFile = null;
}
currentWriteFileOffset = 0;
if (!stopInProgress) {
startStopLock.lock();
try {
compactionThread = null;
startCompactionThread();
} finally {
startStopLock.unlock();
}
}
else {
logger.info("Not restarting thread as the lock is held by stop compaction method.");
}
});
}
@Override
public void run() {
logger.info("Starting compaction thread ...");
int fileToCompact = -1;
while (isRunning) {
try {
fileToCompact = compactionQueue.take();
if (fileToCompact == STOP_SIGNAL) {
logger.debug("Received a stop signal.");
// skip rest of the steps and check status of isRunning flag.
// while pausing/stopping compaction isRunning flag must be set to false.
continue;
}
logger.debug("Compacting {} ...", fileToCompact);
copyFreshRecordsToNewFile(fileToCompact);
logger.debug("Completed compacting {} to {}", fileToCompact, getCurrentWriteFileId());
dbInternal.markFileAsCompacted(fileToCompact);
dbInternal.deleteHaloDBFile(fileToCompact);
}
catch (Exception e) {
logger.error(String.format("Error while compacting file %d to %d", fileToCompact, getCurrentWriteFileId()), e);
}
}
logger.info("Compaction thread stopped.");
}
// TODO: group and move adjacent fresh records together for performance.
private void copyFreshRecordsToNewFile(int idOfFileToCompact) throws IOException {
HaloDBFile fileToCompact = dbInternal.getHaloDBFile(idOfFileToCompact);
if (fileToCompact == null) {
logger.debug("File doesn't exist, was probably compacted already.");
return;
}
FileChannel readFrom = fileToCompact.getChannel();
IndexFile.IndexFileIterator iterator = fileToCompact.getIndexFile().newIterator();
long recordsCopied = 0, recordsScanned = 0;
while (iterator.hasNext()) {
IndexFileEntry indexFileEntry = iterator.next();
byte[] key = indexFileEntry.getKey();
long recordOffset = indexFileEntry.getRecordOffset();
int recordSize = indexFileEntry.getRecordSize();
recordsScanned++;
InMemoryIndexMetaData currentRecordMetaData = dbInternal.getInMemoryIndex().get(key);
if (isRecordFresh(indexFileEntry, currentRecordMetaData, idOfFileToCompact)) {
recordsCopied++;
compactionRateLimiter.acquire(recordSize);
rollOverCurrentWriteFile(recordSize);
sizeOfRecordsCopied += recordSize;
totalSizeOfRecordsCopied += recordSize;
// fresh record, copy to merged file.
long transferred = readFrom.transferTo(recordOffset, recordSize, currentWriteFile.getChannel());
//TODO: for testing. remove.
if (transferred != recordSize) {
logger.error("Had to transfer {} but only did {}", recordSize, transferred);
}
unFlushedData += transferred;
if (dbInternal.options.getFlushDataSizeBytes() != -1 &&
unFlushedData > dbInternal.options.getFlushDataSizeBytes()) {
currentWriteFile.getChannel().force(false);
unFlushedData = 0;
}
IndexFileEntry newEntry = new IndexFileEntry(
key, recordSize, currentWriteFileOffset,
indexFileEntry.getSequenceNumber(), indexFileEntry.getVersion(), -1
);
currentWriteFile.getIndexFile().write(newEntry);
int valueOffset = Utils.getValueOffset(currentWriteFileOffset, key);
InMemoryIndexMetaData newMetaData = new InMemoryIndexMetaData(
currentWriteFile.getFileId(), valueOffset,
currentRecordMetaData.getValueSize(), indexFileEntry.getSequenceNumber()
);
boolean updated = dbInternal.getInMemoryIndex().replace(key, currentRecordMetaData, newMetaData);
if (updated) {
numberOfRecordsReplaced++;
}
else {
// write thread wrote a new version while this version was being compacted.
// therefore, this version is stale.
dbInternal.addFileToCompactionQueueIfThresholdCrossed(currentWriteFile.getFileId(), recordSize);
}
currentWriteFileOffset += recordSize;
currentWriteFile.setWriteOffset(currentWriteFileOffset);
}
}
if (recordsCopied > 0) {
// After compaction we will delete the stale file.
// To prevent data loss in the event of a crash we need to ensure that copied data has hit the disk.
currentWriteFile.flushToDisk();
}
numberOfRecordsCopied += recordsCopied;
numberOfRecordsScanned += recordsScanned;
sizeOfFilesDeleted += fileToCompact.getSize();
logger.debug("Scanned {} records in file {} and copied {} records to {}.datac", recordsScanned, idOfFileToCompact, recordsCopied, getCurrentWriteFileId());
}
private boolean isRecordFresh(IndexFileEntry entry, InMemoryIndexMetaData metaData, int idOfFileToMerge) {
return metaData != null
&& metaData.getFileId() == idOfFileToMerge
&& metaData.getValueOffset() == Utils.getValueOffset(entry.getRecordOffset(), entry.getKey());
}
private void rollOverCurrentWriteFile(int recordSize) throws IOException {
if (currentWriteFile == null || currentWriteFileOffset + recordSize > dbInternal.options
.getMaxFileSize()) {
forceRolloverCurrentWriteFile();
}
}
}
void forceRolloverCurrentWriteFile() throws IOException {
if (currentWriteFile != null) {
currentWriteFile.flushToDisk();
currentWriteFile.getIndexFile().flushToDisk();
}
currentWriteFile = dbInternal.createHaloDBFile(HaloDBFile.FileType.COMPACTED_FILE);
dbInternal.getDbDirectory().syncMetaData();
currentWriteFileOffset = 0;
}
// Used only for tests. to be called only after all writes in the test have been performed.
@VisibleForTesting
synchronized boolean isCompactionComplete() {
if (!isCompactionRunning())
return true;
if (compactionQueue.isEmpty()) {
try {
isRunning = false;
submitFileForCompaction(STOP_SIGNAL);
compactionThread.join();
} catch (InterruptedException e) {
logger.error("Error in isCompactionComplete", e);
}
return true;
}
return false;
}
}
================================================
FILE: src/main/java/com/oath/halodb/Constants.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import java.util.regex.Pattern;
class Constants {
// matches data and compacted files with extension .data and .datac respectively.
static final Pattern DATA_FILE_PATTERN = Pattern.compile("([0-9]+)" + HaloDBFile.DATA_FILE_NAME + "c?");
static final Pattern INDEX_FILE_PATTERN = Pattern.compile("([0-9]+)" + IndexFile.INDEX_FILE_NAME);
static final Pattern TOMBSTONE_FILE_PATTERN = Pattern.compile("([0-9]+)" + TombstoneFile.TOMBSTONE_FILE_NAME);
static final Pattern STORAGE_FILE_PATTERN = Pattern.compile("([0-9]+)\\.[a-z]+");
}
================================================
FILE: src/main/java/com/oath/halodb/DBDirectory.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import java.io.File;
import java.io.IOException;
import java.nio.channels.FileChannel;
import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
import java.util.List;
/**
* Represents the top level directory for a HaloDB instance.
*/
class DBDirectory {
private final File dbDirectory;
private final FileChannel directoryChannel;
private DBDirectory(File dbDirectory, FileChannel directoryChannel) {
this.dbDirectory = dbDirectory;
this.directoryChannel = directoryChannel;
}
/**
* Will create a new directory if one doesn't already exist.
*/
static DBDirectory open(File directory) throws IOException {
FileUtils.createDirectoryIfNotExists(directory);
FileChannel channel = null;
try {
channel = openReadOnlyChannel(directory);
}
catch(IOException e) {
// only swallow the exception if its Windows
if (!isWindows()) {
throw e;
}
}
return new DBDirectory(directory, channel);
}
void close() throws IOException {
if (directoryChannel != null) {
directoryChannel.close();
}
}
Path getPath() {
return dbDirectory.toPath();
}
File[] listDataFiles() {
return FileUtils.listDataFiles(dbDirectory);
}
List listIndexFiles() {
return FileUtils.listIndexFiles(dbDirectory);
}
File[] listTombstoneFiles() {
return FileUtils.listTombstoneFiles(dbDirectory);
}
void syncMetaData() throws IOException {
if (directoryChannel != null) {
directoryChannel.force(true);
}
}
/**
* In Linux the recommended way to flush directory metadata is to open a
* file descriptor for the directory and to call fsync on it. In Java opening a read-only file channel
* and calling force(true) will do the same for us. But this is an undocumented behavior
* in Java and could change in future versions.
* https://grokbase.com/t/lucene/dev/1519kz2s50/recent-java-9-commit-e5b66323ae45-breaks-fsync-on-directory
*
* This currently works on Linux and OSX but may not work on other platforms. Therefore, if there is
* an exception we silently swallow it.
*/
private static FileChannel openReadOnlyChannel(File dbDirectory) throws IOException {
return FileChannel.open(dbDirectory.toPath(), StandardOpenOption.READ);
}
private static boolean isWindows() {
return System.getProperty("os.name", "generic").toLowerCase(java.util.Locale.ENGLISH).indexOf("win") != -1;
}
}
================================================
FILE: src/main/java/com/oath/halodb/DBMetaData.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.nio.channels.SeekableByteChannel;
import java.nio.file.*;
import java.util.zip.CRC32;
import static java.nio.file.StandardOpenOption.*;
import static java.nio.file.StandardCopyOption.*;
/**
* Represents the Metadata for the DB, stored in METADATA_FILE_NAME,
* and contains methods to operate on it.
*/
class DBMetaData {
/**
* checksum - 4 bytes
* version - 1 byte.
* open - 1 byte
* sequence number - 8 bytes.
* io error - 1 byte.
* file size - 4 byte.
*/
private static final int META_DATA_SIZE = 4+1+1+8+1+4;
private static final int CHECK_SUM_SIZE = 4;
private static final int CHECK_SUM_OFFSET = 0;
private long checkSum = 0;
private int version = 0;
private boolean open = false;
private long sequenceNumber = 0;
private boolean ioError = false;
private int maxFileSize = 0;
private final DBDirectory dbDirectory;
static final String METADATA_FILE_NAME = "META";
private static final Object lock = new Object();
DBMetaData(DBDirectory dbDirectory) {
this.dbDirectory = dbDirectory;
}
void loadFromFileIfExists() throws IOException {
synchronized (lock) {
Path metaFile = dbDirectory.getPath().resolve(METADATA_FILE_NAME);
if (Files.exists(metaFile)) {
try (SeekableByteChannel channel = Files.newByteChannel(metaFile)) {
ByteBuffer buff = ByteBuffer.allocate(META_DATA_SIZE);
channel.read(buff);
buff.flip();
checkSum = Utils.toUnsignedIntFromInt(buff.getInt());
version = Utils.toUnsignedByte(buff.get());
open = buff.get() != 0;
sequenceNumber = buff.getLong();
ioError = buff.get() != 0;
maxFileSize = buff.getInt();
}
}
}
}
void storeToFile() throws IOException {
synchronized (lock) {
String tempFileName = METADATA_FILE_NAME + ".temp";
Path tempFile = dbDirectory.getPath().resolve(tempFileName);
Files.deleteIfExists(tempFile);
try(FileChannel channel = FileChannel.open(tempFile, WRITE, CREATE, SYNC)) {
ByteBuffer buff = ByteBuffer.allocate(META_DATA_SIZE);
buff.position(CHECK_SUM_SIZE);
buff.put((byte)version);
buff.put((byte)(open ? 0xFF : 0));
buff.putLong(sequenceNumber);
buff.put((byte)(ioError ? 0xFF : 0));
buff.putInt(maxFileSize);
long crc32 = computeCheckSum(buff.array());
buff.putInt(CHECK_SUM_OFFSET, (int)crc32);
buff.flip();
channel.write(buff);
Files.move(tempFile, dbDirectory.getPath().resolve(METADATA_FILE_NAME), REPLACE_EXISTING, ATOMIC_MOVE);
dbDirectory.syncMetaData();
}
}
}
private long computeCheckSum(byte[] header) {
CRC32 crc32 = new CRC32();
crc32.update(header, CHECK_SUM_OFFSET + CHECK_SUM_SIZE, META_DATA_SIZE - CHECK_SUM_SIZE);
return crc32.getValue();
}
boolean isValid() {
ByteBuffer buff = ByteBuffer.allocate(META_DATA_SIZE);
buff.position(CHECK_SUM_SIZE);
buff.put((byte)version);
buff.put((byte)(open ? 0xFF : 0));
buff.putLong(sequenceNumber);
buff.put((byte)(ioError ? 0xFF : 0));
buff.putInt(maxFileSize);
return computeCheckSum(buff.array()) == checkSum;
}
boolean isOpen() {
return open;
}
void setOpen(boolean open) {
this.open = open;
}
long getSequenceNumber() {
return sequenceNumber;
}
void setSequenceNumber(long sequenceNumber) {
this.sequenceNumber = sequenceNumber;
}
boolean isIOError() {
return ioError;
}
void setIOError(boolean ioError) {
this.ioError = ioError;
}
public int getVersion() {
return version;
}
public void setVersion(int version) {
this.version = version;
}
public int getMaxFileSize() {
return maxFileSize;
}
public void setMaxFileSize(int maxFileSize) {
this.maxFileSize = maxFileSize;
}
}
================================================
FILE: src/main/java/com/oath/halodb/FileUtils.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import java.io.File;
import java.io.IOException;
import java.nio.file.FileSystem;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.spi.FileSystemProvider;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
class FileUtils {
static void createDirectoryIfNotExists(File directory) throws IOException {
if (directory.exists()) {
if (!directory.isDirectory()) {
throw new IOException(directory.getName() + " is not a directory.");
}
return;
}
if (!directory.mkdirs()) {
throw new IOException("Cannot create directory " + directory.getName());
}
}
static void deleteDirectory(File dir) throws IOException {
File[] files = dir.listFiles();
if (files != null) {
for (File file : files) {
if (file.isDirectory()) {
deleteDirectory(file);
} else {
Files.delete(file.toPath());
}
}
}
Files.deleteIfExists(dir.toPath());
}
static List listIndexFiles(File directory) {
File[] files = directory.listFiles(file -> Constants.INDEX_FILE_PATTERN.matcher(file.getName()).matches());
if (files == null)
return Collections.emptyList();
// sort in ascending order. we want the earliest index files to be processed first.
return Arrays.stream(files)
.sorted(Comparator.comparingInt(f -> getFileId(f, Constants.INDEX_FILE_PATTERN)))
.map(f -> getFileId(f, Constants.INDEX_FILE_PATTERN))
.collect(Collectors.toList());
}
/**
* Returns all *.tombstone files in the given directory sorted by file id.
*/
static File[] listTombstoneFiles(File directory) {
File[] files = directory.listFiles(file -> Constants.TOMBSTONE_FILE_PATTERN.matcher(file.getName()).matches());
if (files == null)
return new File[0];
Comparator comparator = Comparator.comparingInt(f -> getFileId(f, Constants.TOMBSTONE_FILE_PATTERN));
Arrays.sort(files, comparator);
return files;
}
/**
* Returns all *.data and *.datac files in the given directory.
*/
static File[] listDataFiles(File directory) {
return directory.listFiles(file -> Constants.DATA_FILE_PATTERN.matcher(file.getName()).matches());
}
private static int getFileId(File file, Pattern pattern) {
Matcher matcher = pattern.matcher(file.getName());
if (matcher.find()) {
return Integer.valueOf(matcher.group(1));
}
throw new IllegalArgumentException("Cannot extract file id for file " + file.getPath());
}
}
================================================
FILE: src/main/java/com/oath/halodb/HaloDB.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import com.google.common.annotations.VisibleForTesting;
import java.io.File;
import java.io.IOException;
import java.util.Set;
public final class HaloDB {
private HaloDBInternal dbInternal;
private File directory;
public static HaloDB open(File dirname, HaloDBOptions opts) throws HaloDBException {
HaloDB db = new HaloDB();
try {
db.dbInternal = HaloDBInternal.open(dirname, opts);
db.directory = dirname;
} catch (IOException e) {
throw new HaloDBException("Failed to open db " + dirname.getName(), e);
}
return db;
}
public static HaloDB open(String directory, HaloDBOptions opts) throws HaloDBException {
return HaloDB.open(new File(directory), opts);
}
public byte[] get(byte[] key) throws HaloDBException {
try {
return dbInternal.get(key, 1);
} catch (IOException e) {
throw new HaloDBException("Lookup failed.", e);
}
}
public boolean put(byte[] key, byte[] value) throws HaloDBException {
try {
return dbInternal.put(key, value);
} catch (IOException e) {
throw new HaloDBException("Store to db failed.", e);
}
}
public void delete(byte[] key) throws HaloDBException {
try {
dbInternal.delete(key);
} catch (IOException e) {
throw new HaloDBException("Delete operation failed.", e);
}
}
public void close() throws HaloDBException {
try {
dbInternal.close();
} catch (IOException e) {
throw new HaloDBException("Error while closing " + directory.getName(), e);
}
}
public long size() {
return dbInternal.size();
}
public HaloDBStats stats() {
return dbInternal.stats();
}
public void resetStats() {
dbInternal.resetStats();
}
public HaloDBIterator newIterator() throws HaloDBException {
return new HaloDBIterator(dbInternal);
}
public HaloDBKeyIterator newKeyIterator() {
return new HaloDBKeyIterator(dbInternal);
}
public void pauseCompaction() throws HaloDBException {
try {
dbInternal.pauseCompaction();
} catch (IOException e) {
throw new HaloDBException("Error while trying to pause compaction thread", e);
}
}
public boolean snapshot() {
return dbInternal.takeSnapshot();
}
public boolean clearSnapshot() {
return dbInternal.clearSnapshot();
}
public File getSnapshotDirectory() {
return dbInternal.getSnapshotDirectory();
}
public void resumeCompaction() {
dbInternal.resumeCompaction();
}
// methods used in tests.
@VisibleForTesting
boolean isCompactionComplete() {
return dbInternal.isCompactionComplete();
}
@VisibleForTesting
boolean isTombstoneFilesMerging() {
return dbInternal.isTombstoneFilesMerging();
}
}
================================================
FILE: src/main/java/com/oath/halodb/HaloDBException.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
public class HaloDBException extends Exception {
private static final long serialVersionUID = 1010101L;
public HaloDBException(String message) {
super(message);
}
public HaloDBException(String message, Throwable cause) {
super(message, cause);
}
public HaloDBException(Throwable cause) {
super(cause);
}
}
================================================
FILE: src/main/java/com/oath/halodb/HaloDBFile.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import com.google.common.primitives.Ints;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Iterator;
import java.util.function.BiFunction;
import java.util.regex.Matcher;
import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
/**
* Represents a data file and its associated index file.
*/
class HaloDBFile {
private static final Logger logger = LoggerFactory.getLogger(HaloDBFile.class);
private volatile int writeOffset;
private FileChannel channel;
private final File backingFile;
private final DBDirectory dbDirectory;
private final int fileId;
private IndexFile indexFile;
private final HaloDBOptions options;
private long unFlushedData = 0;
static final String DATA_FILE_NAME = ".data";
static final String COMPACTED_DATA_FILE_NAME = ".datac";
private final FileType fileType;
private HaloDBFile(int fileId, File backingFile, DBDirectory dbDirectory, IndexFile indexFile, FileType fileType,
FileChannel channel, HaloDBOptions options) throws IOException {
this.fileId = fileId;
this.backingFile = backingFile;
this.dbDirectory = dbDirectory;
this.indexFile = indexFile;
this.fileType = fileType;
this.channel = channel;
this.writeOffset = Ints.checkedCast(channel.size());
this.options = options;
}
byte[] readFromFile(int offset, int length) throws IOException {
byte[] value = new byte[length];
ByteBuffer valueBuf = ByteBuffer.wrap(value);
int read = readFromFile(offset, valueBuf);
assert read == length;
return value;
}
int readFromFile(long position, ByteBuffer destinationBuffer) throws IOException {
long currentPosition = position;
int bytesRead;
do {
bytesRead = channel.read(destinationBuffer, currentPosition);
currentPosition += bytesRead;
} while (bytesRead != -1 && destinationBuffer.hasRemaining());
return (int)(currentPosition - position);
}
private Record readRecord(int offset) throws HaloDBException, IOException {
long tempOffset = offset;
// read the header from disk.
ByteBuffer headerBuf = ByteBuffer.allocate(Record.Header.HEADER_SIZE);
int readSize = readFromFile(offset, headerBuf);
if (readSize != Record.Header.HEADER_SIZE) {
throw new HaloDBException("Corrupted header at " + offset + " in file " + fileId);
}
tempOffset += readSize;
Record.Header header = Record.Header.deserialize(headerBuf);
if (!Record.Header.verifyHeader(header)) {
throw new HaloDBException("Corrupted header at " + offset + " in file " + fileId);
}
// read key-value from disk.
ByteBuffer recordBuf = ByteBuffer.allocate(header.getKeySize() + header.getValueSize());
readSize = readFromFile(tempOffset, recordBuf);
if (readSize != recordBuf.capacity()) {
throw new HaloDBException("Corrupted record at " + offset + " in file " + fileId);
}
Record record = Record.deserialize(recordBuf, header.getKeySize(), header.getValueSize());
record.setHeader(header);
int valueOffset = offset + Record.Header.HEADER_SIZE + header.getKeySize();
record.setRecordMetaData(new InMemoryIndexMetaData(fileId, valueOffset, header.getValueSize(), header.getSequenceNumber()));
return record;
}
InMemoryIndexMetaData writeRecord(Record record) throws IOException {
writeToChannel(record.serialize());
int recordSize = record.getRecordSize();
int recordOffset = writeOffset;
writeOffset += recordSize;
IndexFileEntry indexFileEntry = new IndexFileEntry(
record.getKey(), recordSize,
recordOffset, record.getSequenceNumber(),
Versions.CURRENT_INDEX_FILE_VERSION, -1
);
indexFile.write(indexFileEntry);
int valueOffset = Utils.getValueOffset(recordOffset, record.getKey());
return new InMemoryIndexMetaData(fileId, valueOffset, record.getValue().length, record.getSequenceNumber());
}
void rebuildIndexFile() throws IOException {
indexFile.delete();
indexFile = new IndexFile(fileId, dbDirectory, options);
indexFile.create();
HaloDBFileIterator iterator = new HaloDBFileIterator();
int offset = 0;
while (iterator.hasNext()) {
Record record = iterator.next();
IndexFileEntry indexFileEntry = new IndexFileEntry(
record.getKey(), record.getRecordSize(),
offset, record.getSequenceNumber(),
Versions.CURRENT_INDEX_FILE_VERSION, -1
);
indexFile.write(indexFileEntry);
offset += record.getRecordSize();
}
}
/**
* Copies to a temporary file those records whose computed checksum matches the stored one and then atomically
* rename the temp file to the current file.
* Records in the file which occur after a corrupted record are discarded.
* Index file is also recreated.
* This method is called if we detect an unclean shutdown.
*/
HaloDBFile repairFile(DBDirectory dbDirectory) throws IOException {
HaloDBFile repairFile = createRepairFile();
logger.info("Repairing file {}.", getName());
HaloDBFileIterator iterator = new HaloDBFileIterator();
int count = 0;
while (iterator.hasNext()) {
Record record = iterator.next();
// if the header is corrupted iterator will return null.
if (record != null && record.verifyChecksum()) {
repairFile.writeRecord(record);
count++;
}
else {
logger.info("Found a corrupted record after copying {} records", count);
break;
}
}
logger.info("Recovered {} records from file {} with size {}. Size after repair {}.", count, getName(), getSize(), repairFile.getSize());
repairFile.flushToDisk();
repairFile.indexFile.flushToDisk();
Files.move(repairFile.indexFile.getPath(), indexFile.getPath(), REPLACE_EXISTING, ATOMIC_MOVE);
Files.move(repairFile.getPath(), getPath(), REPLACE_EXISTING, ATOMIC_MOVE);
dbDirectory.syncMetaData();
repairFile.close();
close();
return openForReading(dbDirectory, getPath().toFile(), fileType, options);
}
private HaloDBFile createRepairFile() throws IOException {
File repairFile = dbDirectory.getPath().resolve(getName()+".repair").toFile();
while (!repairFile.createNewFile()) {
logger.info("Repair file {} already exists, probably from a previous repair which failed. Deleting and trying again", repairFile.getName());
repairFile.delete();
}
FileChannel channel = new RandomAccessFile(repairFile, "rw").getChannel();
IndexFile indexFile = new IndexFile(fileId, dbDirectory, options);
indexFile.createRepairFile();
return new HaloDBFile(fileId, repairFile, dbDirectory, indexFile, fileType, channel, options);
}
private long writeToChannel(ByteBuffer[] buffers) throws IOException {
long toWrite = 0;
for (ByteBuffer buffer : buffers) {
toWrite += buffer.remaining();
}
long written = 0;
while (written < toWrite) {
written += channel.write(buffers);
}
unFlushedData += written;
if (options.isSyncWrite() || (options.getFlushDataSizeBytes() != -1 && unFlushedData > options.getFlushDataSizeBytes())) {
flushToDisk();
unFlushedData = 0;
}
return written;
}
void flushToDisk() throws IOException {
if (channel != null && channel.isOpen())
channel.force(true);
}
long getWriteOffset() {
return writeOffset;
}
void setWriteOffset(int writeOffset) {
this.writeOffset = writeOffset;
}
long getSize() {
return writeOffset;
}
IndexFile getIndexFile() {
return indexFile;
}
FileChannel getChannel() {
return channel;
}
FileType getFileType() {
return fileType;
}
int getFileId() {
return fileId;
}
static HaloDBFile openForReading(DBDirectory dbDirectory, File filename, FileType fileType, HaloDBOptions options) throws IOException {
int fileId = HaloDBFile.getFileTimeStamp(filename);
FileChannel channel = new RandomAccessFile(filename, "r").getChannel();
IndexFile indexFile = new IndexFile(fileId, dbDirectory, options);
indexFile.open();
return new HaloDBFile(fileId, filename, dbDirectory, indexFile, fileType, channel, options);
}
static HaloDBFile create(DBDirectory dbDirectory, int fileId, HaloDBOptions options, FileType fileType) throws IOException {
BiFunction toFile = (fileType == FileType.DATA_FILE) ? HaloDBFile::getDataFile : HaloDBFile::getCompactedDataFile;
File file = toFile.apply(dbDirectory, fileId);
while (!file.createNewFile()) {
// file already exists try another one.
fileId++;
file = toFile.apply(dbDirectory, fileId);
}
FileChannel channel = new RandomAccessFile(file, "rw").getChannel();
//TODO: setting the length might improve performance.
//file.setLength(max_);
IndexFile indexFile = new IndexFile(fileId, dbDirectory, options);
indexFile.create();
return new HaloDBFile(fileId, file, dbDirectory, indexFile, fileType, channel, options);
}
HaloDBFileIterator newIterator() throws IOException {
return new HaloDBFileIterator();
}
void close() throws IOException {
if (channel != null) {
channel.close();
}
if (indexFile != null) {
indexFile.close();
}
}
void delete() throws IOException {
close();
if (backingFile != null)
backingFile.delete();
if (indexFile != null)
indexFile.delete();
}
String getName() {
return backingFile.getName();
}
Path getPath() {
return backingFile.toPath();
}
private static File getDataFile(DBDirectory dbDirectory, int fileId) {
return dbDirectory.getPath().resolve(fileId + DATA_FILE_NAME).toFile();
}
private static File getCompactedDataFile(DBDirectory dbDirectory, int fileId) {
return dbDirectory.getPath().resolve(fileId + COMPACTED_DATA_FILE_NAME).toFile();
}
static FileType findFileType(File file) {
String name = file.getName();
return name.endsWith(COMPACTED_DATA_FILE_NAME) ? FileType.COMPACTED_FILE : FileType.DATA_FILE;
}
static int getFileTimeStamp(File file) {
Matcher matcher = Constants.DATA_FILE_PATTERN.matcher(file.getName());
matcher.find();
String s = matcher.group(1);
return Integer.parseInt(s);
}
/**
* This iterator is intended only to be used internally as it behaves bit differently
* from expected Iterator behavior: If a record is corrupted next() will return null although hasNext()
* returns true.
*/
class HaloDBFileIterator implements Iterator {
private final int endOffset;
private int currentOffset = 0;
HaloDBFileIterator() throws IOException {
this.endOffset = Ints.checkedCast(channel.size());
}
@Override
public boolean hasNext() {
return currentOffset < endOffset;
}
@Override
public Record next() {
Record record;
try {
record = readRecord(currentOffset);
} catch (IOException | HaloDBException e) {
// we have encountered an error, probably because record is corrupted.
// we skip rest of the file and return null.
logger.error("Error in iterator", e);
currentOffset = endOffset;
return null;
}
currentOffset += record.getRecordSize();
return record;
}
}
enum FileType {
DATA_FILE, COMPACTED_FILE;
}
}
================================================
FILE: src/main/java/com/oath/halodb/HaloDBInternal.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.primitives.Ints;
import com.google.common.util.concurrent.RateLimiter;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.regex.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.ClosedChannelException;
import java.nio.channels.FileChannel;
import java.nio.channels.FileLock;
import java.nio.channels.OverlappingFileLockException;
import java.nio.file.StandardOpenOption;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
class HaloDBInternal {
private static final Logger logger = LoggerFactory.getLogger(HaloDBInternal.class);
static final String SNAPSHOT_SUBDIR = "snapshot";
private DBDirectory dbDirectory;
private volatile HaloDBFile currentWriteFile;
private volatile TombstoneFile currentTombstoneFile;
private volatile Thread tombstoneMergeThread;
private Map readFileMap = new ConcurrentHashMap<>();
HaloDBOptions options;
private InMemoryIndex inMemoryIndex;
private final Map staleDataPerFileMap = new ConcurrentHashMap<>();
private CompactionManager compactionManager;
private AtomicInteger nextFileId;
private volatile boolean isClosing = false;
private volatile long statsResetTime = System.currentTimeMillis();
private FileLock dbLock;
private final Lock writeLock = new ReentrantLock();
private static final int maxReadAttempts = 5;
private AtomicLong noOfTombstonesCopiedDuringOpen;
private AtomicLong noOfTombstonesFoundDuringOpen;
private volatile long nextSequenceNumber;
private volatile boolean isTombstoneFilesMerging = false;
private HaloDBInternal() {}
static HaloDBInternal open(File directory, HaloDBOptions options) throws HaloDBException, IOException {
checkIfOptionsAreCorrect(options);
HaloDBInternal dbInternal = new HaloDBInternal();
try {
dbInternal.dbDirectory = DBDirectory.open(directory);
dbInternal.dbLock = dbInternal.getLock();
dbInternal.options = options;
int maxFileId = dbInternal.buildReadFileMap();
dbInternal.nextFileId = new AtomicInteger(maxFileId + 10);
dbInternal.noOfTombstonesCopiedDuringOpen = new AtomicLong(0);
dbInternal.noOfTombstonesFoundDuringOpen = new AtomicLong(0);
DBMetaData dbMetaData = new DBMetaData(dbInternal.dbDirectory);
dbMetaData.loadFromFileIfExists();
if (dbMetaData.getMaxFileSize() != 0 && dbMetaData.getMaxFileSize() != options.getMaxFileSize()) {
throw new IllegalArgumentException("File size cannot be changed after db was created. Current size " + dbMetaData.getMaxFileSize());
}
if (dbMetaData.isOpen() || dbMetaData.isIOError()) {
logger.info("DB was not shutdown correctly last time. Files may not be consistent, repairing them.");
// open flag is true, this might mean that the db was not cleanly closed the last time.
dbInternal.repairFiles();
}
dbMetaData.setOpen(true);
dbMetaData.setIOError(false);
dbMetaData.setVersion(Versions.CURRENT_META_FILE_VERSION);
dbMetaData.setMaxFileSize(options.getMaxFileSize());
dbMetaData.storeToFile();
dbInternal.compactionManager = new CompactionManager(dbInternal);
dbInternal.inMemoryIndex = new InMemoryIndex(
options.getNumberOfRecords(), options.isUseMemoryPool(),
options.getFixedKeySize(), options.getMemoryPoolChunkSize()
);
long maxSequenceNumber = dbInternal.buildInMemoryIndex();
if (maxSequenceNumber == -1L) {
dbInternal.nextSequenceNumber = 1;
logger.info("Didn't find any existing records; initializing max sequence number to 1");
} else {
dbInternal.nextSequenceNumber = maxSequenceNumber + 100;
logger.info("Found max sequence number {}, now starting from {}", maxSequenceNumber, dbInternal.nextSequenceNumber);
}
if (!options.isCompactionDisabled()) {
dbInternal.compactionManager.startCompactionThread();
}
else {
logger.warn("Compaction is disabled in HaloDBOption. This should happen only in tests");
}
// merge tombstone files at background if clean up set to true
if (options.isCleanUpTombstonesDuringOpen()) {
dbInternal.isTombstoneFilesMerging = true;
dbInternal.tombstoneMergeThread = new Thread(() -> { dbInternal.mergeTombstoneFiles(); });
dbInternal.tombstoneMergeThread.start();
}
logger.info("Opened HaloDB {}", directory.getName());
logger.info("maxFileSize - {}", options.getMaxFileSize());
logger.info("compactionThresholdPerFile - {}", options.getCompactionThresholdPerFile());
} catch (Exception e) {
// release the lock if open() failed.
if (dbInternal.dbLock != null) {
dbInternal.dbLock.close();
}
throw e;
}
return dbInternal;
}
synchronized void close() throws IOException {
writeLock.lock();
try {
if (isClosing) {
// instance already closed.
return;
}
isClosing = true;
try {
if(!compactionManager.stopCompactionThread(true))
setIOErrorFlag();
} catch (IOException e) {
logger.error("Error while stopping compaction thread. Setting IOError flag", e);
setIOErrorFlag();
}
if (isTombstoneFilesMerging) {
try {
tombstoneMergeThread.join();
} catch (InterruptedException e) {
logger.error("Interrupted when waiting the tombstone files merging");
setIOErrorFlag();
}
}
if (options.isCleanUpInMemoryIndexOnClose())
inMemoryIndex.close();
if (currentWriteFile != null) {
currentWriteFile.flushToDisk();
currentWriteFile.getIndexFile().flushToDisk();
currentWriteFile.close();
}
if (currentTombstoneFile != null) {
currentTombstoneFile.flushToDisk();
currentTombstoneFile.close();
}
for (HaloDBFile file : readFileMap.values()) {
file.close();
}
DBMetaData metaData = new DBMetaData(dbDirectory);
metaData.loadFromFileIfExists();
metaData.setOpen(false);
metaData.storeToFile();
dbDirectory.close();
if (dbLock != null) {
dbLock.close();
}
} finally {
writeLock.unlock();
}
}
boolean put(byte[] key, byte[] value) throws IOException, HaloDBException {
if (key.length > Byte.MAX_VALUE) {
throw new HaloDBException("key length cannot exceed " + Byte.MAX_VALUE);
}
//TODO: more fine-grained locking is possible.
writeLock.lock();
try {
Record record = new Record(key, value);
record.setSequenceNumber(getNextSequenceNumber());
record.setVersion(Versions.CURRENT_DATA_FILE_VERSION);
InMemoryIndexMetaData entry = writeRecordToFile(record);
markPreviousVersionAsStale(key);
//TODO: implement getAndSet and use the return value for
//TODO: markPreviousVersionAsStale method.
return inMemoryIndex.put(key, entry);
} finally {
writeLock.unlock();
}
}
byte[] get(byte[] key, int attemptNumber) throws IOException, HaloDBException {
if (attemptNumber > maxReadAttempts) {
logger.error("Tried {} attempts but read failed", attemptNumber-1);
throw new HaloDBException("Tried " + (attemptNumber-1) + " attempts but failed.");
}
InMemoryIndexMetaData metaData = inMemoryIndex.get(key);
if (metaData == null) {
return null;
}
HaloDBFile readFile = readFileMap.get(metaData.getFileId());
if (readFile == null) {
logger.debug("File {} not present. Compaction job would have deleted it. Retrying ...", metaData.getFileId());
return get(key, attemptNumber+1);
}
try {
return readFile.readFromFile(metaData.getValueOffset(), metaData.getValueSize());
}
catch (ClosedChannelException e) {
if (!isClosing) {
logger.debug("File {} was closed. Compaction job would have deleted it. Retrying ...", metaData.getFileId());
return get(key, attemptNumber+1);
}
// trying to read after HaloDB.close() method called.
throw e;
}
}
int get(byte[] key, ByteBuffer buffer) throws IOException {
InMemoryIndexMetaData metaData = inMemoryIndex.get(key);
if (metaData == null) {
return 0;
}
HaloDBFile readFile = readFileMap.get(metaData.getFileId());
if (readFile == null) {
logger.debug("File {} not present. Compaction job would have deleted it. Retrying ...", metaData.getFileId());
return get(key, buffer);
}
buffer.clear();
buffer.limit(metaData.getValueSize());
try {
int read = readFile.readFromFile(metaData.getValueOffset(), buffer);
buffer.flip();
return read;
}
catch (ClosedChannelException e) {
if (!isClosing) {
logger.debug("File {} was closed. Compaction job would have deleted it. Retrying ...", metaData.getFileId());
return get(key, buffer);
}
// trying to read after HaloDB.close() method called.
throw e;
}
}
//TODO: use fine-grained lock if possible
synchronized boolean takeSnapshot() {
logger.info("Start generating the snapshot");
if (isTombstoneFilesMerging) {
logger.info("DB is merging the tombstone files now. Wait it finished");
try {
tombstoneMergeThread.join();
} catch (InterruptedException e) {
logger.error("Interrupted when waiting the tombstone files merging");
return false;
}
}
try {
final int currentWriteFileId;
compactionManager.pauseCompactionThread();
// Only support one snapshot now
// TODO: support multiple snapshots if needed
File snapshotDir = getSnapshotDirectory();
if (snapshotDir.exists()) {
logger.warn("The snapshot dir is already existed. Delete the old one.");
FileUtils.deleteDirectory(snapshotDir);
}
FileUtils.createDirectoryIfNotExists(snapshotDir);
logger.info("Created directory for snapshot {}", snapshotDir.toString());
writeLock.lock();
try {
forceRollOverCurrentWriteFile();
currentTombstoneFile = forceRollOverTombstoneFile(currentTombstoneFile);
currentWriteFileId = currentWriteFile.getFileId();
} catch (IOException e) {
logger.warn("IO exception when rollover current write files", e);
return false;
} finally {
writeLock.unlock();
}
File[] filesToLink = dbDirectory.getPath().toFile()
.listFiles(file -> {
Matcher m = Constants.STORAGE_FILE_PATTERN.matcher(file.getName());
return m.matches() && (Integer.parseInt(m.group(1)) < currentWriteFileId);
});
compactionManager.forceRolloverCurrentWriteFile();
logger.info("Storage files number need to be linked: {}", filesToLink.length);
for (File file : filesToLink) {
Path dest = Paths.get(snapshotDir.getAbsolutePath(), file.getName());
logger.debug("Create file link from file {} to {}", file.getName(),
dest.toFile().getAbsoluteFile());
Files.createLink(dest, file.toPath());
}
} catch(IOException e) {
logger.warn("IOException when creating snapshot", e);
return false;
} finally {
compactionManager.resumeCompaction();
}
return true;
}
File getSnapshotDirectory() {
Path dbDirectoryPath = dbDirectory.getPath();
return Paths.get(dbDirectoryPath.toFile().getAbsolutePath(), SNAPSHOT_SUBDIR).toFile();
}
boolean clearSnapshot() {
File snapshotDir = getSnapshotDirectory();
if (snapshotDir.exists()) {
try {
FileUtils.deleteDirectory(snapshotDir);
} catch (IOException e) {
logger.error("snapshot deletion error", e);
return false;
}
return true;
} else {
logger.info("snapshot not existed");
return true;
}
}
void delete(byte[] key) throws IOException {
writeLock.lock();
try {
InMemoryIndexMetaData metaData = inMemoryIndex.get(key);
if (metaData != null) {
//TODO: implement a getAndRemove method in InMemoryIndex.
inMemoryIndex.remove(key);
TombstoneEntry entry =
new TombstoneEntry(key, getNextSequenceNumber(), -1, Versions.CURRENT_TOMBSTONE_FILE_VERSION);
currentTombstoneFile = rollOverTombstoneFile(entry, currentTombstoneFile);
currentTombstoneFile.write(entry);
markPreviousVersionAsStale(key, metaData);
}
} finally {
writeLock.unlock();
}
}
long size() {
return inMemoryIndex.size();
}
void setIOErrorFlag() throws IOException {
DBMetaData metaData = new DBMetaData(dbDirectory);
metaData.loadFromFileIfExists();
metaData.setIOError(true);
metaData.storeToFile();
}
void pauseCompaction() throws IOException {
compactionManager.pauseCompactionThread();
}
void resumeCompaction() {
compactionManager.resumeCompaction();
}
private InMemoryIndexMetaData writeRecordToFile(Record record) throws IOException, HaloDBException {
rollOverCurrentWriteFile(record);
return currentWriteFile.writeRecord(record);
}
private void rollOverCurrentWriteFile(Record record) throws IOException {
int size = record.getKey().length + record.getValue().length + Record.Header.HEADER_SIZE;
if ((currentWriteFile == null || currentWriteFile.getWriteOffset() + size > options.getMaxFileSize())
&& !isClosing) {
forceRollOverCurrentWriteFile();
}
}
private void forceRollOverCurrentWriteFile() throws IOException {
if (currentWriteFile != null) {
currentWriteFile.flushToDisk();
currentWriteFile.getIndexFile().flushToDisk();
}
currentWriteFile = createHaloDBFile(HaloDBFile.FileType.DATA_FILE);
dbDirectory.syncMetaData();
}
private TombstoneFile rollOverTombstoneFile(TombstoneEntry entry, TombstoneFile tombstoneFile) throws IOException {
int size = entry.getKey().length + TombstoneEntry.TOMBSTONE_ENTRY_HEADER_SIZE;
if ((tombstoneFile == null ||
tombstoneFile.getWriteOffset() + size > options.getMaxTombstoneFileSize()) && !isClosing) {
tombstoneFile = forceRollOverTombstoneFile(tombstoneFile);
}
return tombstoneFile;
}
private TombstoneFile forceRollOverTombstoneFile(TombstoneFile tombstoneFile) throws IOException {
if (tombstoneFile != null) {
tombstoneFile.flushToDisk();
tombstoneFile.close();
}
tombstoneFile = TombstoneFile.create(dbDirectory, getNextFileId(), options);
dbDirectory.syncMetaData();
return tombstoneFile;
}
private void markPreviousVersionAsStale(byte[] key) {
InMemoryIndexMetaData recordMetaData = inMemoryIndex.get(key);
if (recordMetaData != null) {
markPreviousVersionAsStale(key, recordMetaData);
}
}
private void markPreviousVersionAsStale(byte[] key, InMemoryIndexMetaData recordMetaData) {
int staleRecordSize = Utils.getRecordSize(key.length, recordMetaData.getValueSize());
addFileToCompactionQueueIfThresholdCrossed(recordMetaData.getFileId(), staleRecordSize);
}
void addFileToCompactionQueueIfThresholdCrossed(int fileId, int staleRecordSize) {
HaloDBFile file = readFileMap.get(fileId);
if (file == null)
return;
int staleSizeInFile = updateStaleDataMap(fileId, staleRecordSize);
if (staleSizeInFile >= file.getSize() * options.getCompactionThresholdPerFile()) {
// We don't want to compact the files the writer thread and the compaction thread is currently writing to.
if (getCurrentWriteFileId() != fileId && compactionManager.getCurrentWriteFileId() != fileId) {
if(compactionManager.submitFileForCompaction(fileId)) {
staleDataPerFileMap.remove(fileId);
}
}
}
}
private int updateStaleDataMap(int fileId, int staleDataSize) {
return staleDataPerFileMap.merge(fileId, staleDataSize, (oldValue, newValue) -> oldValue + newValue);
}
void markFileAsCompacted(int fileId) {
staleDataPerFileMap.remove(fileId);
}
InMemoryIndex getInMemoryIndex() {
return inMemoryIndex;
}
HaloDBFile createHaloDBFile(HaloDBFile.FileType fileType) throws IOException {
HaloDBFile file = HaloDBFile.create(dbDirectory, getNextFileId(), options, fileType);
if(readFileMap.putIfAbsent(file.getFileId(), file) != null) {
throw new IOException("Error while trying to create file " + file.getName() + " file with the given id already exists in the map");
}
return file;
}
private List openDataFilesForReading() throws IOException {
File[] files = dbDirectory.listDataFiles();
List result = new ArrayList<>();
for (File f : files) {
HaloDBFile.FileType fileType = HaloDBFile.findFileType(f);
result.add(HaloDBFile.openForReading(dbDirectory, f, fileType, options));
}
return result;
}
/**
* Opens data files for reading and creates a map with file id as the key.
* Also returns the latest file id in the directory which is then used
* to determine the next file id.
*/
private int buildReadFileMap() throws HaloDBException, IOException {
int maxFileId = Integer.MIN_VALUE;
for (HaloDBFile file : openDataFilesForReading()) {
if (readFileMap.putIfAbsent(file.getFileId(), file) != null) {
// There should only be a single file with a given file id.
throw new HaloDBException("Found duplicate file with id " + file.getFileId());
}
maxFileId = Math.max(maxFileId, file.getFileId());
}
if (maxFileId == Integer.MIN_VALUE) {
// no files in the directory. use the current time as the first file id.
maxFileId = Ints.checkedCast(System.currentTimeMillis() / 1000);
}
return maxFileId;
}
private int getNextFileId() {
return nextFileId.incrementAndGet();
}
private Optional getLatestDataFile(HaloDBFile.FileType fileType) {
return readFileMap.values()
.stream()
.filter(f -> f.getFileType() == fileType)
.max(Comparator.comparingInt(HaloDBFile::getFileId));
}
private long buildInMemoryIndex() throws IOException {
int nThreads = options.getBuildIndexThreads();
logger.info("Building index in parallel with {} threads", nThreads);
ExecutorService executor = Executors.newFixedThreadPool(nThreads);
try {
return buildInMemoryIndex(executor);
} finally {
executor.shutdown();
}
}
private long buildInMemoryIndex(ExecutorService executor) throws IOException {
List indexFiles = dbDirectory.listIndexFiles();
logger.info("About to scan {} index files to construct index ...", indexFiles.size());
long start = System.currentTimeMillis();
long maxSequenceNumber = -1l;
List indexFileTasks = new ArrayList<>();
for (int fileId : indexFiles) {
IndexFile indexFile = new IndexFile(fileId, dbDirectory, options);
indexFileTasks.add(new ProcessIndexFileTask(indexFile, fileId));
}
try {
List> results = executor.invokeAll(indexFileTasks);
for (Future result : results) {
maxSequenceNumber = Long.max(result.get(), maxSequenceNumber);
}
} catch (InterruptedException ie) {
throw new IOException("Building index is interrupted");
} catch (ExecutionException ee) {
throw new IOException("Error happened during building in-memory index", ee);
}
logger.info("Completed scanning all index files in {}s", (System.currentTimeMillis() - start) / 1000);
// Scan all the tombstone files and remove records from index.
start = System.currentTimeMillis();
File[] tombStoneFiles = dbDirectory.listTombstoneFiles();
logger.info("About to scan {} tombstone files ...", tombStoneFiles.length);
List tombstoneFileTasks = new ArrayList<>();
for (File file : tombStoneFiles) {
TombstoneFile tombstoneFile = new TombstoneFile(file, options, dbDirectory);
tombstoneFileTasks.add(new ProcessTombstoneFileTask(tombstoneFile));
}
try {
List> results = executor.invokeAll(tombstoneFileTasks);
for (Future result : results) {
maxSequenceNumber = Long.max(result.get(), maxSequenceNumber);
}
} catch (InterruptedException ie) {
throw new IOException("Building index is interrupted");
} catch (ExecutionException ee) {
throw new IOException("Error happened during building in-memory index", ee);
}
logger.info("Completed scanning all tombstone files in {}s", (System.currentTimeMillis() - start) / 1000);
return maxSequenceNumber;
}
class ProcessIndexFileTask implements Callable {
private final IndexFile indexFile;
private final int fileId;
public ProcessIndexFileTask(IndexFile indexFile, int fileId) {
this.indexFile = indexFile;
this.fileId = fileId;
}
@Override
public Long call() throws IOException {
long maxSequenceNumber = -1;
indexFile.open();
IndexFile.IndexFileIterator iterator = indexFile.newIterator();
// build the in-memory index by scanning all index files.
int count = 0, inserted = 0;
while (iterator.hasNext()) {
IndexFileEntry indexFileEntry = iterator.next();
byte[] key = indexFileEntry.getKey();
int recordOffset = indexFileEntry.getRecordOffset();
int recordSize = indexFileEntry.getRecordSize();
long sequenceNumber = indexFileEntry.getSequenceNumber();
maxSequenceNumber = Long.max(sequenceNumber, maxSequenceNumber);
int valueOffset = Utils.getValueOffset(recordOffset, key);
int valueSize = recordSize - (Record.Header.HEADER_SIZE + key.length);
count++;
InMemoryIndexMetaData metaData = new InMemoryIndexMetaData(fileId, valueOffset, valueSize, sequenceNumber);
if (!inMemoryIndex.putIfAbsent(key, metaData)) {
while (true) {
InMemoryIndexMetaData existing = inMemoryIndex.get(key);
if (existing.getSequenceNumber() >= sequenceNumber) {
// stale data, update stale data map.
addFileToCompactionQueueIfThresholdCrossed(fileId, recordSize);
break;
}
if (inMemoryIndex.replace(key, existing, metaData)) {
// update stale data map for the previous version.
addFileToCompactionQueueIfThresholdCrossed(existing.getFileId(), Utils.getRecordSize(key.length, existing.getValueSize()));
inserted++;
break;
}
}
} else {
inserted++;
}
}
logger.debug("Completed scanning index file {}. Found {} records, inserted {} records", fileId, count, inserted);
indexFile.close();
return maxSequenceNumber;
}
}
class ProcessTombstoneFileTask implements Callable {
private final TombstoneFile tombstoneFile;
public ProcessTombstoneFileTask(TombstoneFile tombstoneFile) {
this.tombstoneFile = tombstoneFile;
}
@Override
public Long call() throws IOException {
long maxSequenceNumber = -1;
tombstoneFile.open();
TombstoneFile rolloverFile = null;
TombstoneFile.TombstoneFileIterator iterator = tombstoneFile.newIterator();
long count = 0, active = 0, copied = 0;
while (iterator.hasNext()) {
TombstoneEntry entry = iterator.next();
byte[] key = entry.getKey();
long sequenceNumber = entry.getSequenceNumber();
maxSequenceNumber = Long.max(sequenceNumber, maxSequenceNumber);
count++;
InMemoryIndexMetaData existing = inMemoryIndex.get(key);
if (existing != null && existing.getSequenceNumber() < sequenceNumber) {
// Found a tombstone record which happened after the version currently in index; remove.
inMemoryIndex.remove(key);
// update stale data map for the previous version.
addFileToCompactionQueueIfThresholdCrossed(
existing.getFileId(), Utils.getRecordSize(key.length, existing.getValueSize()));
active++;
if (options.isCleanUpTombstonesDuringOpen()) {
rolloverFile = rollOverTombstoneFile(entry, rolloverFile);
rolloverFile.write(entry);
copied++;
}
}
}
logger.debug("Completed scanning tombstone file {}. Found {} tombstones, {} are still active",
tombstoneFile.getName(), count, active);
tombstoneFile.close();
if (options.isCleanUpTombstonesDuringOpen()) {
logger.debug("Copied {} out of {} tombstones. Deleting {}", copied, count, tombstoneFile.getName());
if (rolloverFile != null) {
logger.debug("Closing rollover tombstone file {}", rolloverFile.getName());
rolloverFile.flushToDisk();
rolloverFile.close();
}
tombstoneFile.delete();
}
noOfTombstonesCopiedDuringOpen.addAndGet(copied);
noOfTombstonesFoundDuringOpen.addAndGet(count);
return maxSequenceNumber;
}
}
HaloDBFile getHaloDBFile(int fileId) {
return readFileMap.get(fileId);
}
void deleteHaloDBFile(int fileId) throws IOException {
HaloDBFile file = readFileMap.get(fileId);
if (file != null) {
readFileMap.remove(fileId);
file.delete();
}
staleDataPerFileMap.remove(fileId);
}
/**
* If options.isCleanUpTombstonesDuringOpen set to true, all inactive entries,
* i.e. physically deleted records, will be dropped during db open.
* Refer to ProcessTombstoneFileTask class and buildInMemoryIndex()
* To shorten db open time, active entries, i.e. not physically deleted
* records, in each tombstone file are rolled over to a corresponding
* new tombstone file. Therefore, the new tombstone file size might be very
* small depends on number of active entries in each tombstone file.
* A tombstone file won't be deleted as long as it has at least 1 active
* entry. This function provide a way to merge small tombstone files in
* offline mode. options.maxTombstoneFileSize still apply to merged file
*/
private void mergeTombstoneFiles() {
File[] tombStoneFiles = dbDirectory.listTombstoneFiles();
logger.info("About to merge {} tombstone files ...", tombStoneFiles.length);
TombstoneFile mergedTombstoneFile = null;
// Use compaction job rate as write rate limiter to avoid IO impact
final RateLimiter rateLimiter = RateLimiter.create(options.getCompactionJobRate());
for (File file : tombStoneFiles) {
TombstoneFile tombstoneFile = new TombstoneFile(file, options, dbDirectory);
if (currentTombstoneFile != null && tombstoneFile.getName().equals(currentTombstoneFile.getName())) {
continue; // not touch current tombstone file
}
try {
tombstoneFile.open();
TombstoneFile.TombstoneFileIterator iterator = tombstoneFile.newIterator();
long count = 0;
while (iterator.hasNext()) {
TombstoneEntry entry = iterator.next();
rateLimiter.acquire(entry.size());
count++;
mergedTombstoneFile = rollOverTombstoneFile(entry, mergedTombstoneFile);
mergedTombstoneFile.write(entry);
}
if (count > 0) {
logger.debug("Merged {} tombstones from {} to {}",
count, tombstoneFile.getName(), mergedTombstoneFile.getName());
}
tombstoneFile.close();
tombstoneFile.delete();
} catch (IOException e) {
logger.error("IO exception when merging tombstone file", e);
}
}
if (mergedTombstoneFile != null) {
try {
mergedTombstoneFile.close();
} catch (IOException e) {
logger.error("IO exception when closing tombstone file: {}", mergedTombstoneFile.getName(), e);
}
}
logger.info("Tombstone files count, before merge:{}, after merge:{}",
tombStoneFiles.length, dbDirectory.listTombstoneFiles().length);
isTombstoneFilesMerging = false;
}
private void repairFiles() {
getLatestDataFile(HaloDBFile.FileType.DATA_FILE).ifPresent(file -> {
try {
logger.info("Repairing file {}.data", file.getFileId());
HaloDBFile repairedFile = file.repairFile(dbDirectory);
readFileMap.put(repairedFile.getFileId(), repairedFile);
}
catch (IOException e) {
throw new RuntimeException("Exception while repairing data file " + file.getFileId() + " which might be corrupted", e);
}
});
getLatestDataFile(HaloDBFile.FileType.COMPACTED_FILE).ifPresent(file -> {
try {
logger.info("Repairing file {}.datac", file.getFileId());
HaloDBFile repairedFile = file.repairFile(dbDirectory);
readFileMap.put(repairedFile.getFileId(), repairedFile);
}
catch (IOException e) {
throw new RuntimeException("Exception while repairing datac file " + file.getFileId() + " which might be corrupted", e);
}
});
File[] tombstoneFiles = dbDirectory.listTombstoneFiles();
if (tombstoneFiles != null && tombstoneFiles.length > 0) {
TombstoneFile lastFile = new TombstoneFile(tombstoneFiles[tombstoneFiles.length-1], options, dbDirectory);
try {
logger.info("Repairing {} file", lastFile.getName());
lastFile.open();
TombstoneFile repairedFile = lastFile.repairFile(dbDirectory);
repairedFile.close();
} catch (IOException e) {
throw new RuntimeException("Exception while repairing tombstone file " + lastFile.getName() + " which might be corrupted", e);
}
}
}
private FileLock getLock() throws HaloDBException {
try {
FileLock lock = FileChannel.open(dbDirectory.getPath().resolve("LOCK"), StandardOpenOption.CREATE, StandardOpenOption.WRITE).tryLock();
if (lock == null) {
logger.error("Error while opening db. Another process already holds a lock to this db.");
throw new HaloDBException("Another process already holds a lock for this db.");
}
return lock;
}
catch (OverlappingFileLockException e) {
logger.error("Error while opening db. Another process already holds a lock to this db.");
throw new HaloDBException("Another process already holds a lock for this db.");
}
catch (IOException e) {
logger.error("Error while trying to get a lock on the db.", e);
throw new HaloDBException("Error while trying to get a lock on the db.", e);
}
}
DBDirectory getDbDirectory() {
return dbDirectory;
}
Set listDataFileIds() {
return new HashSet<>(readFileMap.keySet());
}
boolean isRecordFresh(byte[] key, InMemoryIndexMetaData metaData) {
InMemoryIndexMetaData currentMeta = inMemoryIndex.get(key);
return
currentMeta != null
&&
metaData.getFileId() == currentMeta.getFileId()
&&
metaData.getValueOffset() == currentMeta.getValueOffset();
}
private long getNextSequenceNumber() {
return nextSequenceNumber++;
}
private int getCurrentWriteFileId() {
return currentWriteFile != null ? currentWriteFile.getFileId() : -1;
}
private static void checkIfOptionsAreCorrect(HaloDBOptions options) {
if (options.isUseMemoryPool() && (options.getFixedKeySize() < 0 || options.getFixedKeySize() > Byte.MAX_VALUE)) {
throw new IllegalArgumentException("fixedKeySize must be set and should be less than 128 when using memory pool");
}
}
boolean isClosing() {
return isClosing;
}
HaloDBStats stats() {
OffHeapHashTableStats stats = inMemoryIndex.stats();
return new HaloDBStats(
statsResetTime,
stats.getSize(),
compactionManager.isCompactionRunning(),
compactionManager.noOfFilesPendingCompaction(),
computeStaleDataMapForStats(),
stats.getRehashCount(),
inMemoryIndex.getNoOfSegments(),
inMemoryIndex.getMaxSizeOfEachSegment(),
stats.getSegmentStats(),
dbDirectory.listDataFiles().length,
dbDirectory.listTombstoneFiles().length,
noOfTombstonesFoundDuringOpen.get(),
options.isCleanUpTombstonesDuringOpen() ?
noOfTombstonesFoundDuringOpen.get() - noOfTombstonesCopiedDuringOpen.get() : 0,
compactionManager.getNumberOfRecordsCopied(),
compactionManager.getNumberOfRecordsReplaced(),
compactionManager.getNumberOfRecordsScanned(),
compactionManager.getSizeOfRecordsCopied(),
compactionManager.getSizeOfFilesDeleted(),
compactionManager.getSizeOfFilesDeleted()-compactionManager.getSizeOfRecordsCopied(),
compactionManager.getCompactionJobRateSinceBeginning(),
options.clone()
);
}
synchronized void resetStats() {
inMemoryIndex.resetStats();
compactionManager.resetStats();
statsResetTime = System.currentTimeMillis();
}
private Map computeStaleDataMapForStats() {
Map stats = new HashMap<>();
staleDataPerFileMap.forEach((fileId, staleData) -> {
HaloDBFile file = readFileMap.get(fileId);
if (file != null && file.getSize() > 0) {
double stalePercent = (1.0*staleData/file.getSize()) * 100;
stats.put(fileId, stalePercent);
}
});
return stats;
}
// Used only in tests.
@VisibleForTesting
boolean isCompactionComplete() {
return compactionManager.isCompactionComplete();
}
@VisibleForTesting
boolean isTombstoneFilesMerging() {
return isTombstoneFilesMerging;
}
}
================================================
FILE: src/main/java/com/oath/halodb/HaloDBIterator.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.channels.ClosedChannelException;
import java.util.Iterator;
import java.util.NoSuchElementException;
public class HaloDBIterator implements Iterator {
private static final Logger logger = LoggerFactory.getLogger(HaloDBIterator.class);
private Iterator outer;
private Iterator inner;
private HaloDBFile currentFile;
private Record next;
private final HaloDBInternal dbInternal;
HaloDBIterator(HaloDBInternal dbInternal) {
this.dbInternal = dbInternal;
outer = dbInternal.listDataFileIds().iterator();
}
@Override
public boolean hasNext() {
if (next != null) {
return true;
}
try {
// inner == null means this is the first time hasNext() is called.
// use moveToNextFile() to move to the first file.
if (inner == null && !moveToNextFile()) {
return false;
}
do {
if (readNextRecord()) {
return true;
}
} while (moveToNextFile());
return false;
} catch (IOException e) {
logger.error("Error in Iterator", e);
return false;
}
}
@Override
public Record next() {
if (hasNext()) {
Record record = next;
next = null;
return record;
}
throw new NoSuchElementException();
}
private boolean moveToNextFile() throws IOException {
while (outer.hasNext()) {
int fileId = outer.next();
currentFile = dbInternal.getHaloDBFile(fileId);
if (currentFile != null) {
try {
inner = currentFile.getIndexFile().newIterator();
return true;
} catch (ClosedChannelException e) {
if (dbInternal.isClosing()) {
//TODO: define custom Exception classes for HaloDB.
throw new RuntimeException("DB is closing");
}
logger.debug("Index file {} closed, probably by compaction thread. Skipping to next one", fileId);
}
}
logger.debug("Data file {} deleted, probably by compaction thread. Skipping to next one", fileId);
}
return false;
}
private boolean readNextRecord() {
while (inner.hasNext()) {
IndexFileEntry entry = inner.next();
try {
try {
next = readRecordFromDataFile(entry);
if (next != null) {
return true;
}
} catch (ClosedChannelException e) {
if (dbInternal.isClosing()) {
throw new RuntimeException("DB is closing");
}
logger.debug("Data file {} closed, probably by compaction thread. Skipping to next one", currentFile.getFileId());
break;
}
} catch (IOException e) {
logger.info("Error in iterator", e);
break;
}
}
return false;
}
private Record readRecordFromDataFile(IndexFileEntry entry) throws IOException {
InMemoryIndexMetaData meta = Utils.getMetaData(entry, currentFile.getFileId());
Record record = null;
if (dbInternal.isRecordFresh(entry.getKey(), meta)) {
byte[] value = currentFile.readFromFile(
Utils.getValueOffset(entry.getRecordOffset(), entry.getKey()),
Utils.getValueSize(entry.getRecordSize(), entry.getKey()));
record = new Record(entry.getKey(), value);
record.setRecordMetaData(meta);
}
return record;
}
}
================================================
FILE: src/main/java/com/oath/halodb/HaloDBKeyIterator.java
================================================
package com.oath.halodb;
import java.io.IOException;
import java.nio.channels.ClosedChannelException;
import java.util.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class HaloDBKeyIterator implements Iterator{
private static final Logger logger = LoggerFactory.getLogger(HaloDBIterator.class);
private Iterator outer;
private Iterator inner;
private HaloDBFile currentFile;
private RecordKey next;
private final HaloDBInternal dbInternal;
HaloDBKeyIterator(HaloDBInternal dbInternal) {
this.dbInternal = dbInternal;
outer = dbInternal.listDataFileIds().iterator();
}
@Override
public boolean hasNext() {
if (next != null) {
return true;
}
try {
// inner == null means this is the first time hasNext() is called.
// use moveToNextFile() to move to the first file.
if (inner == null && !moveToNextFile()) {
return false;
}
do {
if (readNextRecord()) {
return true;
}
} while (moveToNextFile());
return false;
} catch (IOException e) {
logger.error("Error in Iterator", e);
return false;
}
}
@Override
public RecordKey next() {
if (hasNext()) {
RecordKey key = next;
next = null;
return key;
}
throw new NoSuchElementException();
}
private boolean moveToNextFile() throws IOException {
while (outer.hasNext()) {
int fileId = outer.next();
currentFile = dbInternal.getHaloDBFile(fileId);
if (currentFile != null) {
try {
inner = currentFile.getIndexFile().newIterator();
return true;
} catch (ClosedChannelException e) {
if (dbInternal.isClosing()) {
//TODO: define custom Exception classes for HaloDB.
throw new RuntimeException("DB is closing");
}
logger.debug("Index file {} closed, probably by compaction thread. Skipping to next one", fileId);
}
}
logger.debug("Data file {} deleted, probably by compaction thread. Skipping to next one", fileId);
}
return false;
}
private boolean readNextRecord() {
while (inner.hasNext()) {
IndexFileEntry entry = inner.next();
try {
try {
next = readValidRecordKey(entry);
if (next != null) {
return true;
}
} catch (ClosedChannelException e) {
if (dbInternal.isClosing()) {
throw new RuntimeException("DB is closing");
}
logger.debug("Data file {} closed, probably by compaction thread. Skipping to next one", currentFile.getFileId());
break;
}
} catch (IOException e) {
logger.info("Error in iterator", e);
break;
}
}
return false;
}
private RecordKey readValidRecordKey(IndexFileEntry entry) throws IOException {
InMemoryIndexMetaData meta = Utils.getMetaData(entry, currentFile.getFileId());
RecordKey key = null;
if (dbInternal.isRecordFresh(entry.getKey(), meta)) {
key = new RecordKey(entry.getKey());
}
return key;
}
}
================================================
FILE: src/main/java/com/oath/halodb/HaloDBOptions.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import com.google.common.base.MoreObjects;
public class HaloDBOptions implements Cloneable {
// threshold of stale data at which file needs to be compacted.
private double compactionThresholdPerFile = 0.75;
private int maxFileSize = 1024 * 1024; /* 1mb file recordSize */
// To keep backward compatibility, initialize to 0 which means
// it will fall back to use maxFileSize, see the getter below
private int maxTombstoneFileSize = 0;
// Data will be flushed to disk after flushDataSizeBytes have been written.
// -1 disables explicit flushing and let the kernel handle it.
private long flushDataSizeBytes = -1;
// Write call will sync data to disk before returning.
// If enabled trades off write throughput for durability.
private boolean syncWrite = false;
private int numberOfRecords = 1_000_000;
// MB of data to be compacted per second.
private int compactionJobRate = 1024 * 1024 * 1024;
private boolean cleanUpInMemoryIndexOnClose = false;
private boolean cleanUpTombstonesDuringOpen = false;
private boolean useMemoryPool = false;
private int fixedKeySize = Byte.MAX_VALUE;
private int memoryPoolChunkSize = 16 * 1024 * 1024;
// Number of threads to scan index and tombstone files
// to build in-memory index at db open
private int buildIndexThreads = 1;
// Just to avoid clients having to deal with CloneNotSupportedException
public HaloDBOptions clone() {
try {
return (HaloDBOptions) super.clone();
} catch (CloneNotSupportedException e) {
return null;
}
}
@Override
public String toString() {
return MoreObjects.toStringHelper("")
.add("compactionThresholdPerFile", compactionThresholdPerFile)
.add("maxFileSize", maxFileSize)
.add("maxTombstoneFileSize", getMaxTombstoneFileSize())
.add("flushDataSizeBytes", flushDataSizeBytes)
.add("syncWrite", syncWrite)
.add("numberOfRecords", numberOfRecords)
.add("compactionJobRate", compactionJobRate)
.add("cleanUpInMemoryIndexOnClose", cleanUpInMemoryIndexOnClose)
.add("cleanUpTombstonesDuringOpen", cleanUpTombstonesDuringOpen)
.add("useMemoryPool", useMemoryPool)
.add("fixedKeySize", fixedKeySize)
.add("memoryPoolChunkSize", memoryPoolChunkSize)
.add("buildIndexThreads", buildIndexThreads)
.toString();
}
public void setCompactionThresholdPerFile(double compactionThresholdPerFile) {
this.compactionThresholdPerFile = compactionThresholdPerFile;
}
public void setMaxFileSize(int maxFileSize) {
if (maxFileSize <= 0) {
throw new IllegalArgumentException("maxFileSize should be > 0");
}
this.maxFileSize = maxFileSize;
}
public void setMaxTombstoneFileSize(int maxFileSize) {
if (maxFileSize <= 0) {
throw new IllegalArgumentException("maxFileSize should be > 0");
}
this.maxTombstoneFileSize = maxFileSize;
}
public void setFlushDataSizeBytes(long flushDataSizeBytes) {
this.flushDataSizeBytes = flushDataSizeBytes;
}
public void setNumberOfRecords(int numberOfRecords) {
this.numberOfRecords = numberOfRecords;
}
public void setCompactionJobRate(int compactionJobRate) {
this.compactionJobRate = compactionJobRate;
}
public void setCleanUpInMemoryIndexOnClose(boolean cleanUpInMemoryIndexOnClose) {
this.cleanUpInMemoryIndexOnClose = cleanUpInMemoryIndexOnClose;
}
public double getCompactionThresholdPerFile() {
return compactionThresholdPerFile;
}
public int getMaxFileSize() {
return maxFileSize;
}
public int getMaxTombstoneFileSize() {
return maxTombstoneFileSize > 0 ? maxTombstoneFileSize : maxFileSize;
}
public long getFlushDataSizeBytes() {
return flushDataSizeBytes;
}
public int getNumberOfRecords() {
return numberOfRecords;
}
public int getCompactionJobRate() {
return compactionJobRate;
}
public boolean isCleanUpInMemoryIndexOnClose() {
return cleanUpInMemoryIndexOnClose;
}
public boolean isCleanUpTombstonesDuringOpen() {
return cleanUpTombstonesDuringOpen;
}
public void setCleanUpTombstonesDuringOpen(boolean cleanUpTombstonesDuringOpen) {
this.cleanUpTombstonesDuringOpen = cleanUpTombstonesDuringOpen;
}
public boolean isUseMemoryPool() {
return useMemoryPool;
}
public void setUseMemoryPool(boolean useMemoryPool) {
this.useMemoryPool = useMemoryPool;
}
public int getFixedKeySize() {
return fixedKeySize;
}
public void setFixedKeySize(int fixedKeySize) {
this.fixedKeySize = fixedKeySize;
}
public int getMemoryPoolChunkSize() {
return memoryPoolChunkSize;
}
public void setMemoryPoolChunkSize(int memoryPoolChunkSize) {
this.memoryPoolChunkSize = memoryPoolChunkSize;
}
public boolean isSyncWrite() {
return syncWrite;
}
public void enableSyncWrites(boolean syncWrites) {
this.syncWrite = syncWrites;
}
public int getBuildIndexThreads() {
return buildIndexThreads;
}
public void setBuildIndexThreads(int buildIndexThreads) {
int numOfProcessors = Runtime.getRuntime().availableProcessors();
if (buildIndexThreads <= 0 || buildIndexThreads > numOfProcessors) {
throw new IllegalArgumentException("buildIndexThreads should be > 0 and <= " + numOfProcessors);
}
this.buildIndexThreads = buildIndexThreads;
}
// to be used only in tests.
private boolean isCompactionDisabled = false;
// not visible to outside the package.
// to be used only in tests.
void setCompactionDisabled(boolean compactionDisabled) {
isCompactionDisabled = compactionDisabled;
}
boolean isCompactionDisabled() {
return isCompactionDisabled;
}
}
================================================
FILE: src/main/java/com/oath/halodb/HaloDBStats.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import com.google.common.base.MoreObjects;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
public class HaloDBStats {
private final long statsResetTime;
private final long size;
private final int numberOfFilesPendingCompaction;
private final Map staleDataPercentPerFile;
private final long rehashCount;
private final long numberOfSegments;
private final long maxSizePerSegment;
private final SegmentStats[] segmentStats;
private final long numberOfTombstonesFoundDuringOpen;
private final long numberOfTombstonesCleanedUpDuringOpen;
private final int numberOfDataFiles;
private final int numberOfTombstoneFiles;
private final long numberOfRecordsCopied;
private final long numberOfRecordsReplaced;
private final long numberOfRecordsScanned;
private final long sizeOfRecordsCopied;
private final long sizeOfFilesDeleted;
private final long sizeReclaimed;
private final long compactionRateInInternal;
private final long compactionRateSinceBeginning;
private final boolean isCompactionRunning;
private final HaloDBOptions options;
public HaloDBStats(long statsResetTime, long size, boolean isCompactionRunning, int numberOfFilesPendingCompaction,
Map staleDataPercentPerFile, long rehashCount, long numberOfSegments,
long maxSizePerSegment, SegmentStats[] segmentStats,
int numberOfDataFiles, int numberOfTombstoneFiles,
long numberOfTombstonesFoundDuringOpen, long numberOfTombstonesCleanedUpDuringOpen,
long numberOfRecordsCopied, long numberOfRecordsReplaced, long numberOfRecordsScanned,
long sizeOfRecordsCopied, long sizeOfFilesDeleted, long sizeReclaimed,
long compactionRateSinceBeginning, HaloDBOptions options) {
this.statsResetTime = statsResetTime;
this.size = size;
this.numberOfFilesPendingCompaction = numberOfFilesPendingCompaction;
this.staleDataPercentPerFile = staleDataPercentPerFile;
this.rehashCount = rehashCount;
this.numberOfSegments = numberOfSegments;
this.maxSizePerSegment = maxSizePerSegment;
this.segmentStats = segmentStats;
this.numberOfDataFiles = numberOfDataFiles;
this.numberOfTombstoneFiles = numberOfTombstoneFiles;
this.numberOfTombstonesFoundDuringOpen = numberOfTombstonesFoundDuringOpen;
this.numberOfTombstonesCleanedUpDuringOpen = numberOfTombstonesCleanedUpDuringOpen;
this.numberOfRecordsCopied = numberOfRecordsCopied;
this.numberOfRecordsReplaced = numberOfRecordsReplaced;
this.numberOfRecordsScanned = numberOfRecordsScanned;
this.sizeOfRecordsCopied = sizeOfRecordsCopied;
this.sizeOfFilesDeleted = sizeOfFilesDeleted;
this.sizeReclaimed = sizeReclaimed;
this.compactionRateSinceBeginning = compactionRateSinceBeginning;
this.isCompactionRunning = isCompactionRunning;
long intervalTimeInSeconds = (System.currentTimeMillis() - statsResetTime)/1000;
if (intervalTimeInSeconds > 0) {
this.compactionRateInInternal = sizeOfRecordsCopied/intervalTimeInSeconds;
}
else {
this.compactionRateInInternal = 0;
}
this.options = options;
}
public long getSize() {
return size;
}
public int getNumberOfFilesPendingCompaction() {
return numberOfFilesPendingCompaction;
}
public Map getStaleDataPercentPerFile() {
return staleDataPercentPerFile;
}
public long getRehashCount() {
return rehashCount;
}
public long getNumberOfSegments() {
return numberOfSegments;
}
public long getMaxSizePerSegment() {
return maxSizePerSegment;
}
public long getNumberOfRecordsCopied() {
return numberOfRecordsCopied;
}
public long getNumberOfRecordsReplaced() {
return numberOfRecordsReplaced;
}
public long getNumberOfRecordsScanned() {
return numberOfRecordsScanned;
}
public long getSizeOfRecordsCopied() {
return sizeOfRecordsCopied;
}
public long getSizeOfFilesDeleted() {
return sizeOfFilesDeleted;
}
public long getSizeReclaimed() {
return sizeReclaimed;
}
public HaloDBOptions getOptions() {
return options;
}
public int getNumberOfDataFiles() {
return numberOfDataFiles;
}
public int getNumberOfTombstoneFiles() {
return numberOfTombstoneFiles;
}
public long getNumberOfTombstonesFoundDuringOpen() {
return numberOfTombstonesFoundDuringOpen;
}
public long getNumberOfTombstonesCleanedUpDuringOpen() {
return numberOfTombstonesCleanedUpDuringOpen;
}
public SegmentStats[] getSegmentStats() {
return segmentStats;
}
public long getCompactionRateInInternal() {
return compactionRateInInternal;
}
public long getCompactionRateSinceBeginning() {
return compactionRateSinceBeginning;
}
public boolean isCompactionRunning() {
return isCompactionRunning;
}
@Override
public String toString() {
return MoreObjects.toStringHelper("")
.add("statsResetTime", statsResetTime)
.add("size", size)
.add("Options", options)
.add("isCompactionRunning", isCompactionRunning)
.add("CompactionJobRateInInterval", getUnit(compactionRateInInternal))
.add("CompactionJobRateSinceBeginning", getUnit(compactionRateSinceBeginning))
.add("numberOfFilesPendingCompaction", numberOfFilesPendingCompaction)
.add("numberOfRecordsCopied", numberOfRecordsCopied)
.add("numberOfRecordsReplaced", numberOfRecordsReplaced)
.add("numberOfRecordsScanned", numberOfRecordsScanned)
.add("sizeOfRecordsCopied", sizeOfRecordsCopied)
.add("sizeOfFilesDeleted", sizeOfFilesDeleted)
.add("sizeReclaimed", sizeReclaimed)
.add("rehashCount", rehashCount)
.add("maxSizePerSegment", maxSizePerSegment)
.add("numberOfDataFiles", numberOfDataFiles)
.add("numberOfTombstoneFiles", numberOfTombstoneFiles)
.add("numberOfTombstonesFoundDuringOpen", numberOfTombstonesFoundDuringOpen)
.add("numberOfTombstonesCleanedUpDuringOpen", numberOfTombstonesCleanedUpDuringOpen)
.add("segmentStats", Arrays.toString(segmentStats))
.add("numberOfSegments", numberOfSegments)
.add("staleDataPercentPerFile", staleDataMapToString())
.toString();
}
public Map toStringMap() {
Map map = new HashMap<>();
map.put("statsResetTime", String.valueOf(statsResetTime));
map.put("size", String.valueOf(size));
map.put("Options", String.valueOf(options));
map.put("isCompactionRunning", String.valueOf(isCompactionRunning));
map.put("CompactionJobRateInInterval", String.valueOf(getUnit(compactionRateInInternal)));
map.put("CompactionJobRateSinceBeginning", String.valueOf(getUnit(compactionRateSinceBeginning)));
map.put("numberOfFilesPendingCompaction", String.valueOf(numberOfFilesPendingCompaction));
map.put("numberOfRecordsCopied", String.valueOf(numberOfRecordsCopied));
map.put("numberOfRecordsReplaced", String.valueOf(numberOfRecordsReplaced));
map.put("numberOfRecordsScanned", String.valueOf(numberOfRecordsScanned));
map.put("sizeOfRecordsCopied", String.valueOf(sizeOfRecordsCopied));
map.put("sizeOfFilesDeleted", String.valueOf(sizeOfFilesDeleted));
map.put("sizeReclaimed", String.valueOf(sizeReclaimed));
map.put("rehashCount", String.valueOf(rehashCount));
map.put("maxSizePerSegment", String.valueOf(maxSizePerSegment));
map.put("numberOfDataFiles", String.valueOf(numberOfDataFiles));
map.put("numberOfTombstoneFiles", String.valueOf(numberOfTombstoneFiles));
map.put("numberOfTombstonesFoundDuringOpen", String.valueOf(numberOfTombstonesFoundDuringOpen));
map.put("numberOfTombstonesCleanedUpDuringOpen", String.valueOf(numberOfTombstonesCleanedUpDuringOpen));
map.put("segmentStats", String.valueOf(Arrays.toString(segmentStats)));
map.put("numberOfSegments", String.valueOf(numberOfSegments));
map.put("staleDataPercentPerFile", String.valueOf(staleDataMapToString()));
return map;
}
private String staleDataMapToString() {
StringBuilder builder = new StringBuilder("[");
boolean isFirst = true;
for (Map.Entry e : staleDataPercentPerFile.entrySet()) {
if (!isFirst) {
builder.append(", ");
}
isFirst = false;
builder.append("{");
builder.append(e.getKey());
builder.append("=");
builder.append(String.format("%.1f", e.getValue()));
builder.append("}");
}
builder.append("]");
return builder.toString();
}
private static final String gbRateUnit = " GB/second";
private static final String mbRateUnit = " MB/second";
private static final String kbRateUnit = " KB/second";
private static final long GB = 1024 * 1024 * 1024;
private static final long MB = 1024 * 1024;
private static final long KB = 1024;
private String getUnit(long value) {
long temp = value / GB;
if (temp >= 1) {
return temp + gbRateUnit;
}
temp = value / MB;
if (temp >= 1) {
return temp + mbRateUnit;
}
temp = value / KB;
return temp + kbRateUnit;
}
}
================================================
FILE: src/main/java/com/oath/halodb/HashAlgorithm.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
enum HashAlgorithm {
MURMUR3,
CRC32,
XX
}
================================================
FILE: src/main/java/com/oath/halodb/HashTableUtil.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
final class HashTableUtil {
// Hash bucket-table
static final long NON_MEMORY_POOL_BUCKET_ENTRY_LEN = 8;
static final long MEMORY_POOL_BUCKET_ENTRY_LEN = 5;
static long allocLen(long keyLen, long valueLen) {
return NonMemoryPoolHashEntries.ENTRY_OFF_DATA + keyLen + valueLen;
}
static int bitNum(long val) {
int bit = 0;
for (; val != 0L; bit++) {
val >>>= 1;
}
return bit;
}
static long roundUpToPowerOf2(long number, long max) {
return number >= max
? max
: (number > 1) ? Long.highestOneBit((number - 1) << 1) : 1;
}
}
================================================
FILE: src/main/java/com/oath/halodb/HashTableValueSerializer.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import java.nio.ByteBuffer;
/**
* Serialize and deserialize cached data using {@link ByteBuffer}
*/
interface HashTableValueSerializer {
void serialize(T value, ByteBuffer buf);
T deserialize(ByteBuffer buf);
int serializedSize(T value);
}
================================================
FILE: src/main/java/com/oath/halodb/Hasher.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import net.jpountz.xxhash.XXHashFactory;
import java.util.zip.CRC32;
abstract class Hasher {
static Hasher create(HashAlgorithm hashAlgorithm) {
String cls = forAlg(hashAlgorithm);
try {
return (Hasher) Class.forName(cls).newInstance();
} catch (ClassNotFoundException e) {
if (hashAlgorithm == HashAlgorithm.XX) {
cls = forAlg(HashAlgorithm.CRC32);
try {
return (Hasher) Class.forName(cls).newInstance();
} catch (InstantiationException | ClassNotFoundException | IllegalAccessException e1) {
throw new RuntimeException(e1);
}
}
throw new RuntimeException(e);
} catch (InstantiationException | IllegalAccessException e) {
throw new RuntimeException(e);
}
}
private static String forAlg(HashAlgorithm hashAlgorithm) {
return Hasher.class.getName()
+ '$'
+ hashAlgorithm.name().substring(0, 1)
+ hashAlgorithm.name().substring(1).toLowerCase()
+ "Hash";
}
abstract long hash(byte[] array);
abstract long hash(long address, long offset, int length);
static final class Crc32Hash extends Hasher {
long hash(byte[] array) {
CRC32 crc = new CRC32();
crc.update(array);
long h = crc.getValue();
h |= h << 32;
return h;
}
long hash(long address, long offset, int length) {
return Uns.crc32(address, offset, length);
}
}
static final class Murmur3Hash extends Hasher {
long hash(byte[] array) {
int o = 0;
int r = array.length;
long h1 = 0L;
long h2 = 0L;
long k1, k2;
for (; r >= 16; r -= 16) {
k1 = getLong(array, o);
o += 8;
k2 = getLong(array, o);
o += 8;
// bmix64()
h1 ^= mixK1(k1);
h1 = Long.rotateLeft(h1, 27);
h1 += h2;
h1 = h1 * 5 + 0x52dce729;
h2 ^= mixK2(k2);
h2 = Long.rotateLeft(h2, 31);
h2 += h1;
h2 = h2 * 5 + 0x38495ab5;
}
if (r > 0) {
k1 = 0;
k2 = 0;
switch (r) {
case 15:
k2 ^= toLong(array[o + 14]) << 48; // fall through
case 14:
k2 ^= toLong(array[o + 13]) << 40; // fall through
case 13:
k2 ^= toLong(array[o + 12]) << 32; // fall through
case 12:
k2 ^= toLong(array[o + 11]) << 24; // fall through
case 11:
k2 ^= toLong(array[o + 10]) << 16; // fall through
case 10:
k2 ^= toLong(array[o + 9]) << 8; // fall through
case 9:
k2 ^= toLong(array[o + 8]); // fall through
case 8:
k1 ^= getLong(array, o);
break;
case 7:
k1 ^= toLong(array[o + 6]) << 48; // fall through
case 6:
k1 ^= toLong(array[o + 5]) << 40; // fall through
case 5:
k1 ^= toLong(array[o + 4]) << 32; // fall through
case 4:
k1 ^= toLong(array[o + 3]) << 24; // fall through
case 3:
k1 ^= toLong(array[o + 2]) << 16; // fall through
case 2:
k1 ^= toLong(array[o + 1]) << 8; // fall through
case 1:
k1 ^= toLong(array[o]);
break;
default:
throw new AssertionError("Should never get here.");
}
h1 ^= mixK1(k1);
h2 ^= mixK2(k2);
}
// makeHash()
h1 ^= array.length;
h2 ^= array.length;
h1 += h2;
h2 += h1;
h1 = fmix64(h1);
h2 = fmix64(h2);
h1 += h2;
//h2 += h1;
// padToLong()
return h1;
}
private static long getLong(byte[] array, int o) {
long l = toLong(array[o + 7]) << 56;
l |= toLong(array[o + 6]) << 48;
l |= toLong(array[o + 5]) << 40;
l |= toLong(array[o + 4]) << 32;
l |= toLong(array[o + 3]) << 24;
l |= toLong(array[o + 2]) << 16;
l |= toLong(array[o + 1]) << 8;
l |= toLong(array[o]);
return l;
}
long hash(long adr, long offset, int length) {
long o = offset;
long r = length;
long h1 = 0L;
long h2 = 0L;
long k1, k2;
for (; r >= 16; r -= 16) {
k1 = getLong(adr, o);
o += 8;
k2 = getLong(adr, o);
o += 8;
// bmix64()
h1 ^= mixK1(k1);
h1 = Long.rotateLeft(h1, 27);
h1 += h2;
h1 = h1 * 5 + 0x52dce729;
h2 ^= mixK2(k2);
h2 = Long.rotateLeft(h2, 31);
h2 += h1;
h2 = h2 * 5 + 0x38495ab5;
}
if (r > 0) {
k1 = 0;
k2 = 0;
switch ((int) r) {
case 15:
k2 ^= toLong(Uns.getByte(adr, o + 14)) << 48; // fall through
case 14:
k2 ^= toLong(Uns.getByte(adr, o + 13)) << 40; // fall through
case 13:
k2 ^= toLong(Uns.getByte(adr, o + 12)) << 32; // fall through
case 12:
k2 ^= toLong(Uns.getByte(adr, o + 11)) << 24; // fall through
case 11:
k2 ^= toLong(Uns.getByte(adr, o + 10)) << 16; // fall through
case 10:
k2 ^= toLong(Uns.getByte(adr, o + 9)) << 8; // fall through
case 9:
k2 ^= toLong(Uns.getByte(adr, o + 8)); // fall through
case 8:
k1 ^= getLong(adr, o);
break;
case 7:
k1 ^= toLong(Uns.getByte(adr, o + 6)) << 48; // fall through
case 6:
k1 ^= toLong(Uns.getByte(adr, o + 5)) << 40; // fall through
case 5:
k1 ^= toLong(Uns.getByte(adr, o + 4)) << 32; // fall through
case 4:
k1 ^= toLong(Uns.getByte(adr, o + 3)) << 24; // fall through
case 3:
k1 ^= toLong(Uns.getByte(adr, o + 2)) << 16; // fall through
case 2:
k1 ^= toLong(Uns.getByte(adr, o + 1)) << 8; // fall through
case 1:
k1 ^= toLong(Uns.getByte(adr, o));
break;
default:
throw new AssertionError("Should never get here.");
}
h1 ^= mixK1(k1);
h2 ^= mixK2(k2);
}
// makeHash()
h1 ^= length;
h2 ^= length;
h1 += h2;
h2 += h1;
h1 = fmix64(h1);
h2 = fmix64(h2);
h1 += h2;
//h2 += h1;
// padToLong()
return h1;
}
private static long getLong(long adr, long o) {
long l = toLong(Uns.getByte(adr, o + 7)) << 56;
l |= toLong(Uns.getByte(adr, o + 6)) << 48;
l |= toLong(Uns.getByte(adr, o + 5)) << 40;
l |= toLong(Uns.getByte(adr, o + 4)) << 32;
l |= toLong(Uns.getByte(adr, o + 3)) << 24;
l |= toLong(Uns.getByte(adr, o + 2)) << 16;
l |= toLong(Uns.getByte(adr, o + 1)) << 8;
l |= toLong(Uns.getByte(adr, o));
return l;
}
static final long C1 = 0x87c37b91114253d5L;
static final long C2 = 0x4cf5ad432745937fL;
static long fmix64(long k) {
k ^= k >>> 33;
k *= 0xff51afd7ed558ccdL;
k ^= k >>> 33;
k *= 0xc4ceb9fe1a85ec53L;
k ^= k >>> 33;
return k;
}
static long mixK1(long k1) {
k1 *= C1;
k1 = Long.rotateLeft(k1, 31);
k1 *= C2;
return k1;
}
static long mixK2(long k2) {
k2 *= C2;
k2 = Long.rotateLeft(k2, 33);
k2 *= C1;
return k2;
}
static long toLong(byte value) {
return value & 0xff;
}
}
static final class XxHash extends Hasher {
private static final XXHashFactory xx = XXHashFactory.fastestInstance();
long hash(long address, long offset, int length) {
return xx.hash64().hash(Uns.directBufferFor(address, offset, length, true), 0);
}
long hash(byte[] array) {
return xx.hash64().hash(array, 0, array.length, 0);
}
}
}
================================================
FILE: src/main/java/com/oath/halodb/InMemoryIndex.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import com.google.common.primitives.Ints;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
/**
* Hash table stored in native memory, outside Java heap.
*/
class InMemoryIndex {
private static final Logger logger = LoggerFactory.getLogger(InMemoryIndex.class);
private final OffHeapHashTable offHeapHashTable;
private final int noOfSegments;
private final int maxSizeOfEachSegment;
InMemoryIndex(int numberOfKeys, boolean useMemoryPool, int fixedKeySize, int memoryPoolChunkSize) {
noOfSegments = Ints.checkedCast(Utils.roundUpToPowerOf2(Runtime.getRuntime().availableProcessors() * 2));
maxSizeOfEachSegment = Ints.checkedCast(Utils.roundUpToPowerOf2(numberOfKeys / noOfSegments));
long start = System.currentTimeMillis();
OffHeapHashTableBuilder builder =
OffHeapHashTableBuilder.newBuilder()
.valueSerializer(new InMemoryIndexMetaDataSerializer())
.segmentCount(noOfSegments)
.hashTableSize(maxSizeOfEachSegment)
.fixedValueSize(InMemoryIndexMetaData.SERIALIZED_SIZE)
.loadFactor(1);
if (useMemoryPool) {
builder.useMemoryPool(true).fixedKeySize(fixedKeySize).memoryPoolChunkSize(memoryPoolChunkSize);
}
this.offHeapHashTable = builder.build();
logger.debug("Allocated memory for the index in {}", (System.currentTimeMillis() - start));
}
boolean put(byte[] key, InMemoryIndexMetaData metaData) {
return offHeapHashTable.put(key, metaData);
}
boolean putIfAbsent(byte[] key, InMemoryIndexMetaData metaData) {
return offHeapHashTable.putIfAbsent(key, metaData);
}
boolean remove(byte[] key) {
return offHeapHashTable.remove(key);
}
boolean replace(byte[] key, InMemoryIndexMetaData oldValue, InMemoryIndexMetaData newValue) {
return offHeapHashTable.addOrReplace(key, oldValue, newValue);
}
InMemoryIndexMetaData get(byte[] key) {
return offHeapHashTable.get(key);
}
boolean containsKey(byte[] key) {
return offHeapHashTable.containsKey(key);
}
void close() {
try {
offHeapHashTable.close();
} catch (IOException e) {
e.printStackTrace();
}
}
long size() {
return offHeapHashTable.size();
}
public OffHeapHashTableStats stats() {
return offHeapHashTable.stats();
}
void resetStats() {
offHeapHashTable.resetStatistics();
}
int getNoOfSegments() {
return noOfSegments;
}
int getMaxSizeOfEachSegment() {
return maxSizeOfEachSegment;
}
}
================================================
FILE: src/main/java/com/oath/halodb/InMemoryIndexMetaData.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import java.nio.ByteBuffer;
/**
* Metadata stored in the in-memory index for each key.
*/
class InMemoryIndexMetaData {
private final int fileId;
private final int valueOffset;
private final int valueSize;
private final long sequenceNumber;
static final int SERIALIZED_SIZE = 4 + 4 + 4 + 8;
InMemoryIndexMetaData(int fileId, int valueOffset, int valueSize, long sequenceNumber) {
this.fileId = fileId;
this.valueOffset = valueOffset;
this.valueSize = valueSize;
this.sequenceNumber = sequenceNumber;
}
void serialize(ByteBuffer byteBuffer) {
byteBuffer.putInt(getFileId());
byteBuffer.putInt(getValueOffset());
byteBuffer.putInt(getValueSize());
byteBuffer.putLong(getSequenceNumber());
byteBuffer.flip();
}
static InMemoryIndexMetaData deserialize(ByteBuffer byteBuffer) {
int fileId = byteBuffer.getInt();
int offset = byteBuffer.getInt();
int size = byteBuffer.getInt();
long sequenceNumber = byteBuffer.getLong();
return new InMemoryIndexMetaData(fileId, offset, size, sequenceNumber);
}
int getFileId() {
return fileId;
}
int getValueOffset() {
return valueOffset;
}
int getValueSize() {
return valueSize;
}
long getSequenceNumber() {
return sequenceNumber;
}
}
================================================
FILE: src/main/java/com/oath/halodb/InMemoryIndexMetaDataSerializer.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import java.nio.ByteBuffer;
class InMemoryIndexMetaDataSerializer implements HashTableValueSerializer {
public void serialize(InMemoryIndexMetaData recordMetaData, ByteBuffer byteBuffer) {
recordMetaData.serialize(byteBuffer);
byteBuffer.flip();
}
public InMemoryIndexMetaData deserialize(ByteBuffer byteBuffer) {
return InMemoryIndexMetaData.deserialize(byteBuffer);
}
public int serializedSize(InMemoryIndexMetaData recordMetaData) {
return InMemoryIndexMetaData.SERIALIZED_SIZE;
}
}
================================================
FILE: src/main/java/com/oath/halodb/IndexFile.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Iterator;
import java.util.Objects;
class IndexFile {
private static final Logger logger = LoggerFactory.getLogger(IndexFile.class);
private final int fileId;
private final DBDirectory dbDirectory;
private File backingFile;
private FileChannel channel;
private final HaloDBOptions options;
private long unFlushedData = 0;
static final String INDEX_FILE_NAME = ".index";
private static final String nullMessage = "Index file entry cannot be null";
IndexFile(int fileId, DBDirectory dbDirectory, HaloDBOptions options) {
this.fileId = fileId;
this.dbDirectory = dbDirectory;
this.options = options;
}
void create() throws IOException {
backingFile = getIndexFile();
if (!backingFile.createNewFile()) {
throw new IOException("Index file with id " + fileId + " already exists");
}
channel = new RandomAccessFile(backingFile, "rw").getChannel();
}
void createRepairFile() throws IOException {
backingFile = getRepairFile();
while (!backingFile.createNewFile()) {
logger.info("Repair file {} already exists, probably from a previous repair which failed. Deleting a trying again", backingFile.getName());
backingFile.delete();
}
channel = new RandomAccessFile(backingFile, "rw").getChannel();
}
void open() throws IOException {
backingFile = getIndexFile();
channel = new RandomAccessFile(backingFile, "rw").getChannel();
}
void close() throws IOException {
if (channel != null) {
channel.close();
}
}
void delete() throws IOException {
if (channel != null && channel.isOpen())
channel.close();
getIndexFile().delete();
}
void write(IndexFileEntry entry) throws IOException {
Objects.requireNonNull(entry, nullMessage);
ByteBuffer[] contents = entry.serialize();
long toWrite = 0;
for (ByteBuffer buffer : contents) {
toWrite += buffer.remaining();
}
long written = 0;
while (written < toWrite) {
written += channel.write(contents);
}
unFlushedData += written;
if (options.getFlushDataSizeBytes() != -1 && unFlushedData > options.getFlushDataSizeBytes()) {
channel.force(false);
unFlushedData = 0;
}
}
void flushToDisk() throws IOException {
if (channel != null && channel.isOpen())
channel.force(true);
}
IndexFileIterator newIterator() throws IOException {
return new IndexFileIterator();
}
Path getPath() {
return backingFile.toPath();
}
private File getIndexFile() {
return dbDirectory.getPath().resolve(fileId + INDEX_FILE_NAME).toFile();
}
private File getRepairFile() {
return dbDirectory.getPath().resolve(fileId + INDEX_FILE_NAME + ".repair").toFile();
}
public class IndexFileIterator implements Iterator {
private final ByteBuffer buffer;
//TODO: index files are not that large, need to check the
// performance since we are memory mapping it.
public IndexFileIterator() throws IOException {
buffer = channel.map(FileChannel.MapMode.READ_ONLY, 0, channel.size());
}
@Override
public boolean hasNext() {
return buffer.hasRemaining();
}
@Override
public IndexFileEntry next() {
if (hasNext()) {
return IndexFileEntry.deserialize(buffer);
}
return null;
}
}
}
================================================
FILE: src/main/java/com/oath/halodb/IndexFileEntry.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import java.nio.ByteBuffer;
import java.util.zip.CRC32;
/**
* This is what is stored in the index file.
*/
class IndexFileEntry {
/**
* checksum - 4 bytes.
* version - 1 byte.
* Key size - 1 bytes.
* record size - 4 bytes.
* record offset - 4 bytes.
* sequence number - 8 bytes
*/
final static int INDEX_FILE_HEADER_SIZE = 22;
final static int CHECKSUM_SIZE = 4;
static final int CHECKSUM_OFFSET = 0;
static final int VERSION_OFFSET = 4;
static final int KEY_SIZE_OFFSET = 5;
static final int RECORD_SIZE_OFFSET = 6;
static final int RECORD_OFFSET = 10;
static final int SEQUENCE_NUMBER_OFFSET = 14;
private final byte[] key;
private final int recordSize;
private final int recordOffset;
private final byte keySize;
private final int version;
private final long sequenceNumber;
private final long checkSum;
IndexFileEntry(byte[] key, int recordSize, int recordOffset, long sequenceNumber, int version, long checkSum) {
this.key = key;
this.recordSize = recordSize;
this.recordOffset = recordOffset;
this.sequenceNumber = sequenceNumber;
this.version = version;
this.checkSum = checkSum;
this.keySize = (byte)key.length;
}
ByteBuffer[] serialize() {
byte[] header = new byte[INDEX_FILE_HEADER_SIZE];
ByteBuffer h = ByteBuffer.wrap(header);
h.put(VERSION_OFFSET, (byte)version);
h.put(KEY_SIZE_OFFSET, keySize);
h.putInt(RECORD_SIZE_OFFSET, recordSize);
h.putInt(RECORD_OFFSET, recordOffset);
h.putLong(SEQUENCE_NUMBER_OFFSET, sequenceNumber);
long crc32 = computeCheckSum(h.array());
h.putInt(CHECKSUM_OFFSET, Utils.toSignedIntFromLong(crc32));
return new ByteBuffer[] { h, ByteBuffer.wrap(key) };
}
static IndexFileEntry deserialize(ByteBuffer buffer) {
long crc32 = Utils.toUnsignedIntFromInt(buffer.getInt());
int version = Utils.toUnsignedByte(buffer.get());
byte keySize = buffer.get();
int recordSize = buffer.getInt();
int offset = buffer.getInt();
long sequenceNumber = buffer.getLong();
byte[] key = new byte[keySize];
buffer.get(key);
return new IndexFileEntry(key, recordSize, offset, sequenceNumber, version, crc32);
}
static IndexFileEntry deserializeIfNotCorrupted(ByteBuffer buffer) {
if (buffer.remaining() < INDEX_FILE_HEADER_SIZE) {
return null;
}
long crc32 = Utils.toUnsignedIntFromInt(buffer.getInt());
int version = Utils.toUnsignedByte(buffer.get());
byte keySize = buffer.get();
int recordSize = buffer.getInt();
int offset = buffer.getInt();
long sequenceNumber = buffer.getLong();
if (sequenceNumber < 0 || keySize <= 0
|| version < 0 || version > 255
|| recordSize <= 0 || offset < 0
|| buffer.remaining() < keySize) {
return null;
}
byte[] key = new byte[keySize];
buffer.get(key);
IndexFileEntry entry = new IndexFileEntry(key, recordSize, offset, sequenceNumber, version, crc32);
if (entry.computeCheckSum() != entry.checkSum) {
return null;
}
return entry;
}
private long computeCheckSum(byte[] header) {
CRC32 crc32 = new CRC32();
crc32.update(header, CHECKSUM_OFFSET + CHECKSUM_SIZE, INDEX_FILE_HEADER_SIZE - CHECKSUM_SIZE);
crc32.update(key);
return crc32.getValue();
}
long computeCheckSum() {
ByteBuffer header = ByteBuffer.allocate(INDEX_FILE_HEADER_SIZE);
header.put(VERSION_OFFSET, (byte)version);
header.put(KEY_SIZE_OFFSET, keySize);
header.putInt(RECORD_SIZE_OFFSET, recordSize);
header.putInt(RECORD_OFFSET, recordOffset);
header.putLong(SEQUENCE_NUMBER_OFFSET, sequenceNumber);
return computeCheckSum(header.array());
}
byte[] getKey() {
return key;
}
int getRecordSize() {
return recordSize;
}
int getRecordOffset() {
return recordOffset;
}
long getSequenceNumber() {
return sequenceNumber;
}
int getVersion() {
return version;
}
long getCheckSum() {
return checkSum;
}
}
================================================
FILE: src/main/java/com/oath/halodb/JNANativeAllocator.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import com.sun.jna.Native;
final class JNANativeAllocator implements NativeMemoryAllocator {
public long allocate(long size) {
try {
return Native.malloc(size);
} catch (OutOfMemoryError oom) {
return 0L;
}
}
public void free(long peer) {
Native.free(peer);
}
public long getTotalAllocated() {
return -1L;
}
}
================================================
FILE: src/main/java/com/oath/halodb/KeyBuffer.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import java.util.Arrays;
final class KeyBuffer {
final byte[] buffer;
private long hash;
KeyBuffer(byte[] buffer) {
this.buffer = buffer;
}
long hash() {
return hash;
}
KeyBuffer finish(Hasher hasher) {
hash = hasher.hash(buffer);
return this;
}
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
KeyBuffer keyBuffer = (KeyBuffer) o;
return buffer.length == keyBuffer.buffer.length && Arrays.equals(keyBuffer.buffer, buffer);
}
public int size() {
return buffer.length;
}
public int hashCode() {
return (int) hash;
}
private static String pad(int val) {
String str = Integer.toHexString(val & 0xff);
while (str.length() == 1) {
str = '0' + str;
}
return str;
}
@Override
public String toString() {
byte[] b = buffer;
StringBuilder sb = new StringBuilder(b.length * 3);
for (int ii = 0; ii < b.length; ii++) {
if (ii % 8 == 0 && ii != 0) {
sb.append('\n');
}
sb.append(pad(b[ii]));
sb.append(' ');
}
return sb.toString();
}
// This is meant to be used only with non-pooled memory.
//TODO: move to another class.
boolean sameKey(long hashEntryAdr) {
long serKeyLen = NonMemoryPoolHashEntries.getKeyLen(hashEntryAdr);
return serKeyLen == buffer.length && compareKey(hashEntryAdr);
}
private boolean compareKey(long hashEntryAdr) {
int blkOff = (int) NonMemoryPoolHashEntries.ENTRY_OFF_DATA;
int p = 0;
int endIdx = buffer.length;
for (; endIdx - p >= 8; p += 8) {
if (Uns.getLong(hashEntryAdr, blkOff + p) != Uns.getLongFromByteArray(buffer, p)) {
return false;
}
}
for (; endIdx - p >= 4; p += 4) {
if (Uns.getInt(hashEntryAdr, blkOff + p) != Uns.getIntFromByteArray(buffer, p)) {
return false;
}
}
for (; endIdx - p >= 2; p += 2) {
if (Uns.getShort(hashEntryAdr, blkOff + p) != Uns.getShortFromByteArray(buffer, p)) {
return false;
}
}
for (; endIdx - p >= 1; p += 1) {
if (Uns.getByte(hashEntryAdr, blkOff + p) != buffer[p]) {
return false;
}
}
return true;
}
}
================================================
FILE: src/main/java/com/oath/halodb/LongArrayList.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import java.util.Arrays;
final class LongArrayList {
private long[] array;
private int size;
public LongArrayList() {
this(10);
}
public LongArrayList(int initialCapacity) {
array = new long[initialCapacity];
}
public long getLong(int i) {
if (i < 0 || i >= size) {
throw new ArrayIndexOutOfBoundsException();
}
return array[i];
}
public void clear() {
size = 0;
}
public int size() {
return size;
}
public void add(long value) {
if (size == array.length) {
array = Arrays.copyOf(array, size * 2);
}
array[size++] = value;
}
}
================================================
FILE: src/main/java/com/oath/halodb/MemoryPoolAddress.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
/**
* Represents the address of an entry in the memory pool. It will have two components: the index of the chunk which
* contains the entry and the offset within the chunk.
*/
class MemoryPoolAddress {
final byte chunkIndex;
final int chunkOffset;
MemoryPoolAddress(byte chunkIndex, int chunkOffset) {
this.chunkIndex = chunkIndex;
this.chunkOffset = chunkOffset;
}
@Override
public boolean equals(Object o) {
if (o == this) {
return true;
}
if (!(o instanceof MemoryPoolAddress)) {
return false;
}
MemoryPoolAddress m = (MemoryPoolAddress) o;
return m.chunkIndex == chunkIndex && m.chunkOffset == chunkOffset;
}
@Override
public int hashCode() {
return 31 * ((31 * chunkIndex) + chunkOffset);
}
}
================================================
FILE: src/main/java/com/oath/halodb/MemoryPoolChunk.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import java.nio.ByteBuffer;
import static com.oath.halodb.MemoryPoolHashEntries.*;
/**
* Memory pool is divided into chunks of configurable size. This represents such a chunk.
*/
class MemoryPoolChunk {
private final long address;
private final int chunkSize;
private final int fixedKeyLength;
private final int fixedValueLength;
private final int fixedSlotSize;
private int writeOffset = 0;
private MemoryPoolChunk(long address, int chunkSize, int fixedKeyLength, int fixedValueLength) {
this.address = address;
this.chunkSize = chunkSize;
this.fixedKeyLength = fixedKeyLength;
this.fixedValueLength = fixedValueLength;
this.fixedSlotSize = HEADER_SIZE + fixedKeyLength + fixedValueLength;
}
static MemoryPoolChunk create(int chunkSize, int fixedKeyLength, int fixedValueLength) {
int fixedSlotSize = HEADER_SIZE + fixedKeyLength + fixedValueLength;
if (fixedSlotSize > chunkSize) {
throw new IllegalArgumentException("fixedSlotSize " + fixedSlotSize + " must be smaller than chunkSize " + chunkSize);
}
long address = Uns.allocate(chunkSize, true);
return new MemoryPoolChunk(address, chunkSize, fixedKeyLength, fixedValueLength);
}
void destroy() {
Uns.free(address);
}
MemoryPoolAddress getNextAddress(int slotOffset) {
byte chunkIndex = Uns.getByte(address, slotOffset + ENTRY_OFF_NEXT_CHUNK_INDEX);
int chunkOffset = Uns.getInt(address, slotOffset + ENTRY_OFF_NEXT_CHUNK_OFFSET);
return new MemoryPoolAddress(chunkIndex, chunkOffset);
}
void setNextAddress(int slotOffset, MemoryPoolAddress next) {
Uns.putByte(address, slotOffset + ENTRY_OFF_NEXT_CHUNK_INDEX, next.chunkIndex);
Uns.putInt(address, slotOffset + ENTRY_OFF_NEXT_CHUNK_OFFSET, next.chunkOffset);
}
/**
* Relative put method. Writes to the slot pointed to by the writeOffset and increments the writeOffset.
*/
void fillNextSlot(byte[] key, byte[] value, MemoryPoolAddress nextAddress) {
fillSlot(writeOffset, key, value, nextAddress);
writeOffset += fixedSlotSize;
}
/**
* Absolute put method. Writes to the slot pointed to by the offset.
*/
void fillSlot(int slotOffset, byte[] key, byte[] value, MemoryPoolAddress nextAddress) {
if (key.length > fixedKeyLength || value.length != fixedValueLength) {
throw new IllegalArgumentException(
String.format("Invalid request. Key length %d. fixed key length %d. Value length %d",
key.length, fixedKeyLength, value.length)
);
}
if (chunkSize - slotOffset < fixedSlotSize) {
throw new IllegalArgumentException(
String.format("Invalid offset %d. Chunk size %d. fixed slot size %d",
slotOffset, chunkSize, fixedSlotSize)
);
}
setNextAddress(slotOffset, nextAddress);
Uns.putByte(address, slotOffset + ENTRY_OFF_KEY_LENGTH, (byte) key.length);
Uns.copyMemory(key, 0, address, slotOffset + ENTRY_OFF_DATA, key.length);
setValue(value, slotOffset);
}
void setValue(byte[] value, int slotOffset) {
if (value.length != fixedValueLength) {
throw new IllegalArgumentException(
String.format("Invalid value length. fixedValueLength %d, value length %d",
fixedValueLength, value.length)
);
}
Uns.copyMemory(value, 0, address, slotOffset + ENTRY_OFF_DATA + fixedKeyLength, value.length);
}
int getWriteOffset() {
return writeOffset;
}
int remaining() {
return chunkSize - writeOffset;
}
ByteBuffer readOnlyValueByteBuffer(int offset) {
return Uns.directBufferFor(address, offset + ENTRY_OFF_DATA + fixedKeyLength, fixedValueLength, true);
}
ByteBuffer readOnlyKeyByteBuffer(int offset) {
return Uns.directBufferFor(address, offset + ENTRY_OFF_DATA, getKeyLength(offset), true);
}
long computeHash(int slotOffset, Hasher hasher) {
return hasher.hash(address, slotOffset + ENTRY_OFF_DATA, getKeyLength(slotOffset));
}
boolean compareKey(int slotOffset, byte[] key) {
if (key.length > fixedKeyLength || slotOffset + fixedSlotSize > chunkSize) {
throw new IllegalArgumentException("Invalid request. slotOffset - " + slotOffset + " key.length - " + key.length);
}
return getKeyLength(slotOffset) == key.length && compare(slotOffset + ENTRY_OFF_DATA, key);
}
boolean compareValue(int slotOffset, byte[] value) {
if (value.length != fixedValueLength || slotOffset + fixedSlotSize > chunkSize) {
throw new IllegalArgumentException("Invalid request. slotOffset - " + slotOffset + " value.length - " + value.length);
}
return compare(slotOffset + ENTRY_OFF_DATA + fixedKeyLength, value);
}
private boolean compare(int offset, byte[] array) {
int p = 0, length = array.length;
for (; length - p >= 8; p += 8) {
if (Uns.getLong(address, offset + p) != Uns.getLongFromByteArray(array, p)) {
return false;
}
}
for (; length - p >= 4; p += 4) {
if (Uns.getInt(address, offset + p) != Uns.getIntFromByteArray(array, p)) {
return false;
}
}
for (; length - p >= 2; p += 2) {
if (Uns.getShort(address, offset + p) != Uns.getShortFromByteArray(array, p)) {
return false;
}
}
for (; length - p >= 1; p += 1) {
if (Uns.getByte(address, offset + p) != array[p]) {
return false;
}
}
return true;
}
private byte getKeyLength(int slotOffset) {
return Uns.getByte(address, slotOffset + ENTRY_OFF_KEY_LENGTH);
}
}
================================================
FILE: src/main/java/com/oath/halodb/MemoryPoolHashEntries.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
class MemoryPoolHashEntries {
/*
* chunk index - 1 byte.
* chunk offset - 4 byte.
* key length - 1 byte.
*/
static final int HEADER_SIZE = 1 + 4 + 1;
static final int ENTRY_OFF_NEXT_CHUNK_INDEX = 0;
static final int ENTRY_OFF_NEXT_CHUNK_OFFSET = 1;
// offset of key length (1 bytes, byte)
static final int ENTRY_OFF_KEY_LENGTH = 5;
// offset of data in first block
static final int ENTRY_OFF_DATA = 6;
}
================================================
FILE: src/main/java/com/oath/halodb/NativeMemoryAllocator.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
//This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
interface NativeMemoryAllocator {
long allocate(long size);
void free(long peer);
long getTotalAllocated();
}
================================================
FILE: src/main/java/com/oath/halodb/NonMemoryPoolHashEntries.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
/**
* Encapsulates access to hash entries.
*/
final class NonMemoryPoolHashEntries {
// offset of next hash entry in a hash bucket (8 bytes, long)
static final long ENTRY_OFF_NEXT = 0;
// offset of key length (1 bytes, byte)
static final long ENTRY_OFF_KEY_LENGTH = 8;
// offset of data in first block
static final long ENTRY_OFF_DATA = 9;
static void init(int keyLen, long hashEntryAdr) {
setNext(hashEntryAdr, 0L);
Uns.putByte(hashEntryAdr, ENTRY_OFF_KEY_LENGTH, (byte) keyLen);
}
static long getNext(long hashEntryAdr) {
return hashEntryAdr != 0L ? Uns.getLong(hashEntryAdr, ENTRY_OFF_NEXT) : 0L;
}
static void setNext(long hashEntryAdr, long nextAdr) {
if (hashEntryAdr == nextAdr) {
throw new IllegalArgumentException();
}
if (hashEntryAdr != 0L) {
Uns.putLong(hashEntryAdr, ENTRY_OFF_NEXT, nextAdr);
}
}
static int getKeyLen(long hashEntryAdr) {
return Uns.getByte(hashEntryAdr, ENTRY_OFF_KEY_LENGTH);
}
}
================================================
FILE: src/main/java/com/oath/halodb/OffHeapHashTable.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import com.oath.halodb.histo.EstimatedHistogram;
import java.io.Closeable;
interface OffHeapHashTable extends Closeable {
/**
* @param key key of the entry to be added. Must not be {@code null}.
* @param value value of the entry to be added. Must not be {@code null}.
* @return {@code true}, if the entry has been added, {@code false} otherwise
*/
boolean put(byte[] key, V value);
/**
* Adds key/value if either the key is not present and {@code old} is null or the existing value matches parameter {@code old}.
*
* @param key key of the entry to be added or replaced. Must not be {@code null}.
* @param old if the entry exists, it's serialized value is compared to the serialized value of {@code old}
* and only replaced, if it matches.
* @param value value of the entry to be added. Must not be {@code null}.
* @return {@code true} on success or {@code false} if the existing value does not matcuh {@code old}
*/
boolean addOrReplace(byte[] key, V old, V value);
/**
* @param key key of the entry to be added. Must not be {@code null}.
* @param value value of the entry to be added. Must not be {@code null}.
* @return {@code true} on success or {@code false} if the key is already present.
*/
boolean putIfAbsent(byte[] key, V value);
/**
* Remove a single entry for the given key.
*
* @param key key of the entry to be removed. Must not be {@code null}.
* @return {@code true}, if the entry has been removed, {@code false} otherwise
*/
boolean remove(byte[] key);
/**
* Removes all entries from the cache.
*/
void clear();
/**
* Get the value for a given key.
*
* @param key key of the entry to be retrieved. Must not be {@code null}.
* @return either the non-{@code null} value or {@code null} if no entry for the requested key exists
*/
V get(byte[] key);
/**
* Checks whether an entry for a given key exists.
* Usually, this is more efficient than testing for {@code null} via {@link #get(Object)}.
*
* @param key key of the entry to be retrieved. Must not be {@code null}.
* @return either {@code true} if an entry for the given key exists or {@code false} if no entry for the requested key exists
*/
boolean containsKey(byte[] key);
// statistics / information
void resetStatistics();
long size();
int[] hashTableSizes();
SegmentStats[] perSegmentStats();
EstimatedHistogram getBucketHistogram();
int segments();
float loadFactor();
OffHeapHashTableStats stats();
}
================================================
FILE: src/main/java/com/oath/halodb/OffHeapHashTableBuilder.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
class OffHeapHashTableBuilder {
private int segmentCount;
private int hashTableSize = 8192;
private int memoryPoolChunkSize = 2 * 1024 * 1024;
private HashTableValueSerializer valueSerializer;
private float loadFactor = .75f;
private int fixedKeySize = -1;
private int fixedValueSize = -1;
private HashAlgorithm hashAlgorighm = HashAlgorithm.MURMUR3;
private Hasher hasher;
private boolean unlocked;
private boolean useMemoryPool = false;
private OffHeapHashTableBuilder() {
int cpus = Runtime.getRuntime().availableProcessors();
segmentCount = roundUpToPowerOf2(cpus * 2, 1 << 30);
}
static final String SYSTEM_PROPERTY_PREFIX = "org.caffinitas.ohc.";
static int roundUpToPowerOf2(int number, int max) {
return number >= max
? max
: (number > 1) ? Integer.highestOneBit((number - 1) << 1) : 1;
}
static OffHeapHashTableBuilder newBuilder() {
return new OffHeapHashTableBuilder<>();
}
public OffHeapHashTable build() {
if (fixedValueSize == -1) {
throw new IllegalArgumentException("Need to set fixedValueSize");
}
//TODO: write a test.
if (useMemoryPool && fixedKeySize == -1) {
throw new IllegalArgumentException("Need to set fixedKeySize when using memory pool");
}
if (valueSerializer == null) {
throw new IllegalArgumentException("Value serializer must be set.");
}
return new OffHeapHashTableImpl<>(this);
}
public int getHashTableSize() {
return hashTableSize;
}
public OffHeapHashTableBuilder hashTableSize(int hashTableSize) {
if (hashTableSize < -1) {
throw new IllegalArgumentException("hashTableSize:" + hashTableSize);
}
this.hashTableSize = hashTableSize;
return this;
}
public int getMemoryPoolChunkSize() {
return memoryPoolChunkSize;
}
public OffHeapHashTableBuilder memoryPoolChunkSize(int chunkSize) {
if (chunkSize < -1) {
throw new IllegalArgumentException("memoryPoolChunkSize:" + chunkSize);
}
this.memoryPoolChunkSize = chunkSize;
return this;
}
public HashTableValueSerializer getValueSerializer() {
return valueSerializer;
}
public OffHeapHashTableBuilder valueSerializer(HashTableValueSerializer valueSerializer) {
this.valueSerializer = valueSerializer;
return this;
}
public int getSegmentCount() {
return segmentCount;
}
public OffHeapHashTableBuilder segmentCount(int segmentCount) {
if (segmentCount < -1) {
throw new IllegalArgumentException("segmentCount:" + segmentCount);
}
this.segmentCount = segmentCount;
return this;
}
public float getLoadFactor() {
return loadFactor;
}
public OffHeapHashTableBuilder loadFactor(float loadFactor) {
if (loadFactor <= 0f) {
throw new IllegalArgumentException("loadFactor:" + loadFactor);
}
this.loadFactor = loadFactor;
return this;
}
public int getFixedKeySize() {
return fixedKeySize;
}
public OffHeapHashTableBuilder fixedKeySize(int fixedKeySize) {
if (fixedKeySize <= 0) {
throw new IllegalArgumentException("fixedValueSize:" + fixedKeySize);
}
this.fixedKeySize = fixedKeySize;
return this;
}
public int getFixedValueSize() {
return fixedValueSize;
}
public OffHeapHashTableBuilder fixedValueSize(int fixedValueSize) {
if (fixedValueSize <= 0) {
throw new IllegalArgumentException("fixedValueSize:" + fixedValueSize);
}
this.fixedValueSize = fixedValueSize;
return this;
}
public HashAlgorithm getHashAlgorighm() {
return hashAlgorighm;
}
public Hasher getHasher() {
return hasher;
}
public OffHeapHashTableBuilder hashMode(HashAlgorithm hashMode) {
if (hashMode == null) {
throw new NullPointerException("hashMode");
}
this.hashAlgorighm = hashMode;
this.hasher = Hasher.create(hashMode);
return this;
}
public boolean isUnlocked() {
return unlocked;
}
public OffHeapHashTableBuilder unlocked(boolean unlocked) {
this.unlocked = unlocked;
return this;
}
public boolean isUseMemoryPool() {
return useMemoryPool;
}
public OffHeapHashTableBuilder useMemoryPool(boolean useMemoryPool) {
this.useMemoryPool = useMemoryPool;
return this;
}
}
================================================
FILE: src/main/java/com/oath/halodb/OffHeapHashTableImpl.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import com.google.common.primitives.Ints;
import com.oath.halodb.histo.EstimatedHistogram;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
final class OffHeapHashTableImpl implements OffHeapHashTable {
private static final Logger logger = LoggerFactory.getLogger(OffHeapHashTableImpl.class);
private final HashTableValueSerializer valueSerializer;
private final int fixedValueLength;
private final List> segments;
private final long segmentMask;
private final int segmentShift;
private final int segmentCount;
private volatile long putFailCount;
private boolean closed;
private final Hasher hasher;
OffHeapHashTableImpl(OffHeapHashTableBuilder builder) {
this.hasher = Hasher.create(builder.getHashAlgorighm());
this.fixedValueLength = builder.getFixedValueSize();
// build segments
if (builder.getSegmentCount() <= 0) {
throw new IllegalArgumentException("Segment count should be > 0");
}
segmentCount = Ints.checkedCast(HashTableUtil.roundUpToPowerOf2(builder.getSegmentCount(), 1 << 30));
segments = new ArrayList<>(segmentCount);
for (int i = 0; i < segmentCount; i++) {
try {
segments.add(allocateSegment(builder));
} catch (RuntimeException e) {
for (; i >= 0; i--) {
if (segments.get(i) != null) {
segments.get(i).release();
}
}
throw e;
}
}
// bit-mask for segment part of hash
int bitNum = HashTableUtil.bitNum(segmentCount) - 1;
this.segmentShift = 64 - bitNum;
this.segmentMask = ((long) segmentCount - 1) << segmentShift;
this.valueSerializer = builder.getValueSerializer();
if (valueSerializer == null) {
throw new NullPointerException("valueSerializer == null");
}
logger.debug("off-heap index with {} segments created.", segmentCount);
}
private Segment allocateSegment(OffHeapHashTableBuilder builder) {
if (builder.isUseMemoryPool()) {
return new SegmentWithMemoryPool<>(builder);
}
return new SegmentNonMemoryPool<>(builder);
}
public V get(byte[] key) {
if (key == null) {
throw new NullPointerException();
}
KeyBuffer keySource = keySource(key);
return segment(keySource.hash()).getEntry(keySource);
}
public boolean containsKey(byte[] key) {
if (key == null) {
throw new NullPointerException();
}
KeyBuffer keySource = keySource(key);
return segment(keySource.hash()).containsEntry(keySource);
}
public boolean put(byte[] k, V v) {
return putInternal(k, v, false, null);
}
public boolean addOrReplace(byte[] key, V old, V value) {
return putInternal(key, value, false, old);
}
public boolean putIfAbsent(byte[] k, V v) {
return putInternal(k, v, true, null);
}
private boolean putInternal(byte[] key, V value, boolean ifAbsent, V old) {
if (key == null || value == null) {
throw new NullPointerException();
}
int valueSize = valueSize(value);
if (valueSize != fixedValueLength) {
throw new IllegalArgumentException("value size " + valueSize + " greater than fixed value size " + fixedValueLength);
}
if (old != null && valueSize(old) != fixedValueLength) {
throw new IllegalArgumentException("old value size " + valueSize(old) + " greater than fixed value size " + fixedValueLength);
}
if (key.length > Byte.MAX_VALUE) {
throw new IllegalArgumentException("key size of " + key.length + " exceeds max permitted size of " + Byte.MAX_VALUE);
}
long hash = hasher.hash(key);
return segment(hash).putEntry(key, value, hash, ifAbsent, old);
}
private int valueSize(V v) {
int sz = valueSerializer.serializedSize(v);
if (sz <= 0) {
throw new IllegalArgumentException("Illegal value length " + sz);
}
return sz;
}
public boolean remove(byte[] k) {
if (k == null) {
throw new NullPointerException();
}
KeyBuffer keySource = keySource(k);
return segment(keySource.hash()).removeEntry(keySource);
}
private Segment segment(long hash) {
int seg = (int) ((hash & segmentMask) >>> segmentShift);
return segments.get(seg);
}
private KeyBuffer keySource(byte[] key) {
KeyBuffer keyBuffer = new KeyBuffer(key);
return keyBuffer.finish(hasher);
}
//
// maintenance
//
public void clear() {
for (Segment map : segments) {
map.clear();
}
}
//
// state
//
//TODO: remove.
public void setCapacity(long capacity) {
}
public void close() {
closed = true;
for (Segment map : segments) {
map.release();
}
Collections.fill(segments, null);
if (logger.isDebugEnabled()) {
logger.debug("Closing OHC instance");
}
}
//
// statistics and related stuff
//
public void resetStatistics() {
for (Segment map : segments) {
map.resetStatistics();
}
putFailCount = 0;
}
public OffHeapHashTableStats stats() {
long hitCount = 0, missCount = 0, size = 0,
freeCapacity = 0, rehashes = 0, putAddCount = 0, putReplaceCount = 0, removeCount = 0;
for (Segment map : segments) {
hitCount += map.hitCount();
missCount += map.missCount();
size += map.size();
rehashes += map.rehashes();
putAddCount += map.putAddCount();
putReplaceCount += map.putReplaceCount();
removeCount += map.removeCount();
}
return new OffHeapHashTableStats(
hitCount,
missCount,
size,
rehashes,
putAddCount,
putReplaceCount,
putFailCount,
removeCount,
perSegmentStats());
}
public long size() {
long size = 0L;
for (Segment map : segments) {
size += map.size();
}
return size;
}
public int segments() {
return segments.size();
}
public float loadFactor() {
return segments.get(0).loadFactor();
}
public int[] hashTableSizes() {
int[] r = new int[segments.size()];
for (int i = 0; i < segments.size(); i++) {
r[i] = segments.get(i).hashTableSize();
}
return r;
}
public long[] perSegmentSizes() {
long[] r = new long[segments.size()];
for (int i = 0; i < segments.size(); i++) {
r[i] = segments.get(i).size();
}
return r;
}
public SegmentStats[] perSegmentStats() {
SegmentStats[] stats = new SegmentStats[segments.size()];
for (int i = 0; i < stats.length; i++) {
Segment map = segments.get(i);
stats[i] = new SegmentStats(map.size(), map.numberOfChunks(), map.numberOfSlots(), map.freeListSize());
}
return stats;
}
public EstimatedHistogram getBucketHistogram() {
EstimatedHistogram hist = new EstimatedHistogram();
for (Segment map : segments) {
map.updateBucketHistogram(hist);
}
long[] offsets = hist.getBucketOffsets();
long[] buckets = hist.getBuckets(false);
for (int i = buckets.length - 1; i > 0; i--) {
if (buckets[i] != 0L) {
offsets = Arrays.copyOf(offsets, i + 2);
buckets = Arrays.copyOf(buckets, i + 3);
System.arraycopy(offsets, 0, offsets, 1, i + 1);
System.arraycopy(buckets, 0, buckets, 1, i + 2);
offsets[0] = 0L;
buckets[0] = 0L;
break;
}
}
for (int i = 0; i < offsets.length; i++) {
offsets[i]--;
}
return new EstimatedHistogram(offsets, buckets);
}
public String toString() {
return getClass().getSimpleName() + " ,segments=" + segments.size();
}
}
================================================
FILE: src/main/java/com/oath/halodb/OffHeapHashTableStats.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import com.google.common.base.Objects;
final class OffHeapHashTableStats {
private final long hitCount;
private final long missCount;
private final long size;
private final long rehashCount;
private final long putAddCount;
private final long putReplaceCount;
private final long putFailCount;
private final long removeCount;
private final SegmentStats[] segmentStats;
public OffHeapHashTableStats(long hitCount, long missCount,
long size, long rehashCount,
long putAddCount, long putReplaceCount, long putFailCount, long removeCount,
SegmentStats[] segmentStats) {
this.hitCount = hitCount;
this.missCount = missCount;
this.size = size;
this.rehashCount = rehashCount;
this.putAddCount = putAddCount;
this.putReplaceCount = putReplaceCount;
this.putFailCount = putFailCount;
this.removeCount = removeCount;
this.segmentStats = segmentStats;
}
public long getRehashCount() {
return rehashCount;
}
public long getHitCount() {
return hitCount;
}
public long getMissCount() {
return missCount;
}
public long getSize() {
return size;
}
public long getPutAddCount() {
return putAddCount;
}
public long getPutReplaceCount() {
return putReplaceCount;
}
public long getPutFailCount() {
return putFailCount;
}
public long getRemoveCount() {
return removeCount;
}
public SegmentStats[] getSegmentStats() {
return segmentStats;
}
public String toString() {
return Objects.toStringHelper(this)
.add("hitCount", hitCount)
.add("missCount", missCount)
.add("size", size)
.add("rehashCount", rehashCount)
.add("put(add/replace/fail)", Long.toString(putAddCount) + '/' + putReplaceCount + '/' + putFailCount)
.add("removeCount", removeCount)
.toString();
}
private static long maxOf(long[] arr) {
long r = 0;
for (long l : arr) {
if (l > r) {
r = l;
}
}
return r;
}
private static long minOf(long[] arr) {
long r = Long.MAX_VALUE;
for (long l : arr) {
if (l < r) {
r = l;
}
}
return r;
}
private static double avgOf(long[] arr) {
double r = 0d;
for (long l : arr) {
r += l;
}
return r / arr.length;
}
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
OffHeapHashTableStats that = (OffHeapHashTableStats) o;
if (hitCount != that.hitCount) return false;
if (missCount != that.missCount) return false;
if (putAddCount != that.putAddCount) return false;
if (putFailCount != that.putFailCount) return false;
if (putReplaceCount != that.putReplaceCount) return false;
// if (rehashCount != that.rehashCount) return false;
if (removeCount != that.removeCount) return false;
if (size != that.size) return false;
// if (totalAllocated != that.totalAllocated) return false;
return true;
}
public int hashCode() {
int result = (int) (hitCount ^ (hitCount >>> 32));
result = 31 * result + (int) (missCount ^ (missCount >>> 32));
result = 31 * result + (int) (size ^ (size >>> 32));
// result = 31 * result + (int) (rehashCount ^ (rehashCount >>> 32));
result = 31 * result + (int) (putAddCount ^ (putAddCount >>> 32));
result = 31 * result + (int) (putReplaceCount ^ (putReplaceCount >>> 32));
result = 31 * result + (int) (putFailCount ^ (putFailCount >>> 32));
result = 31 * result + (int) (removeCount ^ (removeCount >>> 32));
// result = 31 * result + (int) (totalAllocated ^ (totalAllocated >>> 32));
return result;
}
}
================================================
FILE: src/main/java/com/oath/halodb/Record.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.zip.CRC32;
public class Record {
private final byte[] key, value;
private InMemoryIndexMetaData recordMetaData;
private Header header;
public Record(byte[] key, byte[] value) {
this.key = key;
this.value = value;
header = new Header(0, Versions.CURRENT_DATA_FILE_VERSION, (byte)key.length, value.length, -1);
}
ByteBuffer[] serialize() {
ByteBuffer headerBuf = serializeHeaderAndComputeChecksum();
return new ByteBuffer[] {headerBuf, ByteBuffer.wrap(key), ByteBuffer.wrap(value)};
}
static Record deserialize(ByteBuffer buffer, short keySize, int valueSize) {
buffer.flip();
byte[] key = new byte[keySize];
byte[] value = new byte[valueSize];
buffer.get(key);
buffer.get(value);
return new Record(key, value);
}
public byte[] getKey() {
return key;
}
public byte[] getValue() {
return value;
}
InMemoryIndexMetaData getRecordMetaData() {
return recordMetaData;
}
void setRecordMetaData(InMemoryIndexMetaData recordMetaData) {
this.recordMetaData = recordMetaData;
}
/**
* @return recordSize which is HEADER_SIZE + key size + value size.
*/
int getRecordSize() {
return header.getRecordSize();
}
void setSequenceNumber(long sequenceNumber) {
header.sequenceNumber = sequenceNumber;
}
long getSequenceNumber() {
return header.getSequenceNumber();
}
void setVersion(int version) {
if (version < 0 || version > 255) {
throw new IllegalArgumentException("Got version " + version + ". Record version must be in range [0,255]");
}
header.version = version;
}
int getVersion() {
return header.version;
}
Header getHeader() {
return header;
}
void setHeader(Header header) {
this.header = header;
}
private ByteBuffer serializeHeaderAndComputeChecksum() {
ByteBuffer headerBuf = header.serialize();
long checkSum = computeCheckSum(headerBuf.array());
headerBuf.putInt(Header.CHECKSUM_OFFSET, Utils.toSignedIntFromLong(checkSum));
return headerBuf;
}
boolean verifyChecksum() {
ByteBuffer headerBuf = header.serialize();
long checkSum = computeCheckSum(headerBuf.array());
return checkSum == header.getCheckSum();
}
private long computeCheckSum(byte[] header) {
CRC32 crc32 = new CRC32();
// compute checksum with all but the first header element, key and value.
crc32.update(header, Header.CHECKSUM_OFFSET + Header.CHECKSUM_SIZE, Header.HEADER_SIZE-Header.CHECKSUM_SIZE);
crc32.update(key);
crc32.update(value);
return crc32.getValue();
}
@Override
public boolean equals(Object obj) {
// to be used in tests as we don't check if the headers are the same.
if (this == obj) {
return true;
}
if (!(obj instanceof Record)) {
return false;
}
Record record = (Record)obj;
return Arrays.equals(getKey(), record.getKey()) && Arrays.equals(getValue(), record.getValue());
}
static class Header {
/**
* crc - 4 bytes.
* version - 1 byte.
* key size - 1 bytes.
* value size - 4 bytes.
* sequence number - 8 bytes.
*/
static final int CHECKSUM_OFFSET = 0;
static final int VERSION_OFFSET = 4;
static final int KEY_SIZE_OFFSET = 5;
static final int VALUE_SIZE_OFFSET = 6;
static final int SEQUENCE_NUMBER_OFFSET = 10;
static final int HEADER_SIZE = 18;
static final int CHECKSUM_SIZE = 4;
private long checkSum;
private int version;
private byte keySize;
private int valueSize;
private long sequenceNumber;
private int recordSize;
Header(long checkSum, int version, byte keySize, int valueSize, long sequenceNumber) {
this.checkSum = checkSum;
this.version = version;
this.keySize = keySize;
this.valueSize = valueSize;
this.sequenceNumber = sequenceNumber;
recordSize = keySize + valueSize + HEADER_SIZE;
}
static Header deserialize(ByteBuffer buffer) {
long checkSum = Utils.toUnsignedIntFromInt(buffer.getInt(CHECKSUM_OFFSET));
int version = Utils.toUnsignedByte(buffer.get(VERSION_OFFSET));
byte keySize = buffer.get(KEY_SIZE_OFFSET);
int valueSize = buffer.getInt(VALUE_SIZE_OFFSET);
long sequenceNumber = buffer.getLong(SEQUENCE_NUMBER_OFFSET);
return new Header(checkSum, version, keySize, valueSize, sequenceNumber);
}
// checksum value can be computed only with record key and value.
ByteBuffer serialize() {
byte[] header = new byte[HEADER_SIZE];
ByteBuffer headerBuffer = ByteBuffer.wrap(header);
headerBuffer.put(VERSION_OFFSET, (byte)version);
headerBuffer.put(KEY_SIZE_OFFSET, keySize);
headerBuffer.putInt(VALUE_SIZE_OFFSET, valueSize);
headerBuffer.putLong(SEQUENCE_NUMBER_OFFSET, sequenceNumber);
return headerBuffer;
}
static boolean verifyHeader(Record.Header header) {
return header.version >= 0 && header.version < 256
&& header.keySize > 0 && header.valueSize > 0
&& header.recordSize > 0 && header.sequenceNumber > 0;
}
byte getKeySize() {
return keySize;
}
int getValueSize() {
return valueSize;
}
int getRecordSize() {
return recordSize;
}
long getSequenceNumber() {
return sequenceNumber;
}
long getCheckSum() {
return checkSum;
}
int getVersion() {
return version;
}
}
}
================================================
FILE: src/main/java/com/oath/halodb/RecordKey.java
================================================
package com.oath.halodb;
import java.util.*;
public class RecordKey {
final byte[] key;
public RecordKey(byte[] key) {
this.key = key;
}
public byte[] getBytes() {
return key;
}
@Override
public boolean equals(Object obj) {
// to be used in tests as we don't check if the headers are the same.
if (this == obj) {
return true;
}
if (!(obj instanceof RecordKey)) {
return false;
}
RecordKey recordKey = (RecordKey)obj;
return Arrays.equals(this.key, recordKey.getBytes());
}
}
================================================
FILE: src/main/java/com/oath/halodb/Segment.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import com.oath.halodb.histo.EstimatedHistogram;
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
abstract class Segment {
final HashTableValueSerializer valueSerializer;
final int fixedValueLength;
final int fixedKeyLength;
private final Hasher hasher;
private volatile long lock;
private static final AtomicLongFieldUpdater lockFieldUpdater =
AtomicLongFieldUpdater.newUpdater(Segment.class, "lock");
Segment(HashTableValueSerializer valueSerializer, int fixedValueLength, Hasher hasher) {
this(valueSerializer, fixedValueLength, -1, hasher);
}
Segment(HashTableValueSerializer valueSerializer, int fixedValueLength, int fixedKeyLength, Hasher hasher) {
this.valueSerializer = valueSerializer;
this.fixedValueLength = fixedValueLength;
this.fixedKeyLength = fixedKeyLength;
this.hasher = hasher;
}
boolean lock() {
long t = Thread.currentThread().getId();
if (t == lockFieldUpdater.get(this)) {
return false;
}
while (true) {
if (lockFieldUpdater.compareAndSet(this, 0L, t)) {
return true;
}
// yield control to other thread.
// Note: we cannot use LockSupport.parkNanos() as that does not
// provide nanosecond resolution on Windows.
Thread.yield();
}
}
void unlock(boolean wasFirst) {
if (!wasFirst) {
return;
}
long t = Thread.currentThread().getId();
boolean r = lockFieldUpdater.compareAndSet(this, t, 0L);
assert r;
}
KeyBuffer keySource(byte[] key) {
KeyBuffer keyBuffer = new KeyBuffer(key);
return keyBuffer.finish(hasher);
}
abstract V getEntry(KeyBuffer key);
abstract boolean containsEntry(KeyBuffer key);
abstract boolean putEntry(byte[] key, V value, long hash, boolean ifAbsent, V oldValue);
abstract boolean removeEntry(KeyBuffer key);
abstract long size();
abstract void release();
abstract void clear();
abstract long hitCount();
abstract long missCount();
abstract long putAddCount();
abstract long putReplaceCount();
abstract long removeCount();
abstract void resetStatistics();
abstract long rehashes();
abstract float loadFactor();
abstract int hashTableSize();
abstract void updateBucketHistogram(EstimatedHistogram hist);
//Used only in memory pool.
long numberOfChunks() {
return -1;
}
long numberOfSlots() {
return -1;
}
long freeListSize() {
return -1;
}
}
================================================
FILE: src/main/java/com/oath/halodb/SegmentNonMemoryPool.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import com.google.common.primitives.Ints;
import com.oath.halodb.histo.EstimatedHistogram;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
class SegmentNonMemoryPool extends Segment {
private static final Logger logger = LoggerFactory.getLogger(SegmentNonMemoryPool.class);
// maximum hash table size
private static final int MAX_TABLE_SIZE = 1 << 30;
long size;
Table table;
private long hitCount;
private long missCount;
private long putAddCount;
private long putReplaceCount;
private long removeCount;
private long threshold;
private final float loadFactor;
private long rehashes;
long evictedEntries;
private final HashAlgorithm hashAlgorithm;
private static final boolean throwOOME = true;
SegmentNonMemoryPool(OffHeapHashTableBuilder builder) {
super(builder.getValueSerializer(), builder.getFixedValueSize(), builder.getHasher());
this.hashAlgorithm = builder.getHashAlgorighm();
int hts = builder.getHashTableSize();
if (hts <= 0) {
hts = 8192;
}
if (hts < 256) {
hts = 256;
}
int msz = Ints.checkedCast(HashTableUtil.roundUpToPowerOf2(hts, MAX_TABLE_SIZE));
table = Table.create(msz, throwOOME);
if (table == null) {
throw new RuntimeException("unable to allocate off-heap memory for segment");
}
float lf = builder.getLoadFactor();
if (lf <= .0d) {
lf = .75f;
}
this.loadFactor = lf;
threshold = (long) ((double) table.size() * loadFactor);
}
@Override
void release() {
boolean wasFirst = lock();
try {
table.release();
table = null;
} finally {
unlock(wasFirst);
}
}
@Override
long size() {
return size;
}
@Override
long hitCount() {
return hitCount;
}
@Override
long missCount() {
return missCount;
}
@Override
long putAddCount() {
return putAddCount;
}
@Override
long putReplaceCount() {
return putReplaceCount;
}
@Override
long removeCount() {
return removeCount;
}
@Override
void resetStatistics() {
rehashes = 0L;
evictedEntries = 0L;
hitCount = 0L;
missCount = 0L;
putAddCount = 0L;
putReplaceCount = 0L;
removeCount = 0L;
}
@Override
long rehashes() {
return rehashes;
}
@Override
V getEntry(KeyBuffer key) {
boolean wasFirst = lock();
try {
for (long hashEntryAdr = table.getFirst(key.hash());
hashEntryAdr != 0L;
hashEntryAdr = NonMemoryPoolHashEntries.getNext(hashEntryAdr)) {
if (key.sameKey(hashEntryAdr)) {
hitCount++;
return valueSerializer.deserialize(Uns.readOnlyBuffer(hashEntryAdr, fixedValueLength, NonMemoryPoolHashEntries.ENTRY_OFF_DATA + NonMemoryPoolHashEntries.getKeyLen(hashEntryAdr)));
}
}
missCount++;
return null;
} finally {
unlock(wasFirst);
}
}
@Override
boolean containsEntry(KeyBuffer key) {
boolean wasFirst = lock();
try {
for (long hashEntryAdr = table.getFirst(key.hash());
hashEntryAdr != 0L;
hashEntryAdr = NonMemoryPoolHashEntries.getNext(hashEntryAdr)) {
if (key.sameKey(hashEntryAdr)) {
hitCount++;
return true;
}
}
missCount++;
return false;
} finally {
unlock(wasFirst);
}
}
@Override
boolean putEntry(byte[] key, V value, long hash, boolean ifAbsent, V oldValue) {
long oldValueAdr = 0L;
try {
if (oldValue != null) {
oldValueAdr = Uns.allocate(fixedValueLength, throwOOME);
if (oldValueAdr == 0L) {
throw new RuntimeException("Unable to allocate " + fixedValueLength + " bytes in off-heap");
}
valueSerializer.serialize(oldValue, Uns.directBufferFor(oldValueAdr, 0, fixedValueLength, false));
}
long hashEntryAdr;
if ((hashEntryAdr = Uns.allocate(HashTableUtil.allocLen(key.length, fixedValueLength), throwOOME)) == 0L) {
// entry too large to be inserted or OS is not able to provide enough memory
removeEntry(keySource(key));
return false;
}
// initialize hash entry
NonMemoryPoolHashEntries.init(key.length, hashEntryAdr);
serializeForPut(key, value, hashEntryAdr);
if (putEntry(hashEntryAdr, hash, key.length, ifAbsent, oldValueAdr)) {
return true;
}
Uns.free(hashEntryAdr);
return false;
} finally {
Uns.free(oldValueAdr);
}
}
private boolean putEntry(long newHashEntryAdr, long hash, long keyLen, boolean putIfAbsent, long oldValueAddr) {
long removeHashEntryAdr = 0L;
boolean wasFirst = lock();
try {
long hashEntryAdr;
long prevEntryAdr = 0L;
for (hashEntryAdr = table.getFirst(hash);
hashEntryAdr != 0L;
prevEntryAdr = hashEntryAdr, hashEntryAdr = NonMemoryPoolHashEntries.getNext(hashEntryAdr)) {
if (notSameKey(newHashEntryAdr, hash, keyLen, hashEntryAdr)) {
continue;
}
// putIfAbsent is true, but key is already present, return.
if (putIfAbsent) {
return false;
}
// key already exists, we just need to replace the value.
if (oldValueAddr != 0L) {
// code for replace() operation
if (!Uns.memoryCompare(hashEntryAdr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA + keyLen, oldValueAddr, 0L, fixedValueLength)) {
return false;
}
}
removeInternal(hashEntryAdr, prevEntryAdr, hash);
removeHashEntryAdr = hashEntryAdr;
break;
}
// key is not present in the map, therefore we need to add a new entry.
if (hashEntryAdr == 0L) {
// key is not present but old value is not null.
// we consider this as a mismatch and return.
if (oldValueAddr != 0) {
return false;
}
if (size >= threshold) {
rehash();
}
size++;
}
add(newHashEntryAdr, hash);
if (hashEntryAdr == 0L) {
putAddCount++;
} else {
putReplaceCount++;
}
return true;
} finally {
unlock(wasFirst);
if (removeHashEntryAdr != 0L) {
Uns.free(removeHashEntryAdr);
}
}
}
private static boolean notSameKey(long newHashEntryAdr, long newHash, long newKeyLen, long hashEntryAdr) {
long serKeyLen = NonMemoryPoolHashEntries.getKeyLen(hashEntryAdr);
return serKeyLen != newKeyLen
|| !Uns.memoryCompare(hashEntryAdr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA, newHashEntryAdr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA, serKeyLen);
}
private void serializeForPut(byte[] key, V value, long hashEntryAdr) {
try {
Uns.buffer(hashEntryAdr, key.length, NonMemoryPoolHashEntries.ENTRY_OFF_DATA).put(key);
if (value != null) {
valueSerializer.serialize(value, Uns.buffer(hashEntryAdr, fixedValueLength, NonMemoryPoolHashEntries.ENTRY_OFF_DATA + key.length));
}
} catch (Throwable e) {
freeAndThrow(e, hashEntryAdr);
}
}
private void freeAndThrow(Throwable e, long hashEntryAdr) {
Uns.free(hashEntryAdr);
if (e instanceof RuntimeException) {
throw (RuntimeException) e;
}
if (e instanceof Error) {
throw (Error) e;
}
throw new RuntimeException(e);
}
@Override
void clear() {
boolean wasFirst = lock();
try {
size = 0L;
long next;
for (int p = 0; p < table.size(); p++) {
for (long hashEntryAdr = table.getFirst(p);
hashEntryAdr != 0L;
hashEntryAdr = next) {
next = NonMemoryPoolHashEntries.getNext(hashEntryAdr);
Uns.free(hashEntryAdr);
}
}
table.clear();
} finally {
unlock(wasFirst);
}
}
@Override
boolean removeEntry(KeyBuffer key) {
long removeHashEntryAdr = 0L;
boolean wasFirst = lock();
try {
long prevEntryAdr = 0L;
for (long hashEntryAdr = table.getFirst(key.hash());
hashEntryAdr != 0L;
prevEntryAdr = hashEntryAdr, hashEntryAdr = NonMemoryPoolHashEntries.getNext(hashEntryAdr)) {
if (!key.sameKey(hashEntryAdr)) {
continue;
}
// remove existing entry
removeHashEntryAdr = hashEntryAdr;
removeInternal(hashEntryAdr, prevEntryAdr, key.hash());
size--;
removeCount++;
return true;
}
return false;
} finally {
unlock(wasFirst);
if (removeHashEntryAdr != 0L) {
Uns.free(removeHashEntryAdr);
}
}
}
private void rehash() {
long start = System.currentTimeMillis();
Table tab = table;
int tableSize = tab.size();
if (tableSize > MAX_TABLE_SIZE) {
// already at max hash table size
return;
}
Table newTable = Table.create(tableSize * 2, throwOOME);
if (newTable == null) {
return;
}
long next;
Hasher hasher = Hasher.create(hashAlgorithm);
for (int part = 0; part < tableSize; part++) {
for (long hashEntryAdr = tab.getFirst(part);
hashEntryAdr != 0L;
hashEntryAdr = next) {
next = NonMemoryPoolHashEntries.getNext(hashEntryAdr);
NonMemoryPoolHashEntries.setNext(hashEntryAdr, 0L);
long hash = hasher.hash(hashEntryAdr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA, NonMemoryPoolHashEntries.getKeyLen(hashEntryAdr));
newTable.addAsHead(hash, hashEntryAdr);
}
}
threshold = (long) ((float) newTable.size() * loadFactor);
table.release();
table = newTable;
rehashes++;
logger.info("Completed rehashing segment in {} ms.", (System.currentTimeMillis() - start));
}
float loadFactor() {
return loadFactor;
}
int hashTableSize() {
return table.size();
}
void updateBucketHistogram(EstimatedHistogram hist) {
boolean wasFirst = lock();
try {
table.updateBucketHistogram(hist);
} finally {
unlock(wasFirst);
}
}
void getEntryAddresses(int mapSegmentIndex, int nSegments, LongArrayList hashEntryAdrs) {
boolean wasFirst = lock();
try {
for (; nSegments-- > 0 && mapSegmentIndex < table.size(); mapSegmentIndex++) {
for (long hashEntryAdr = table.getFirst(mapSegmentIndex);
hashEntryAdr != 0L;
hashEntryAdr = NonMemoryPoolHashEntries.getNext(hashEntryAdr)) {
hashEntryAdrs.add(hashEntryAdr);
}
}
} finally {
unlock(wasFirst);
}
}
static final class Table {
final int mask;
final long address;
private boolean released;
static Table create(int hashTableSize, boolean throwOOME) {
int msz = Ints.checkedCast(HashTableUtil.NON_MEMORY_POOL_BUCKET_ENTRY_LEN * hashTableSize);
long address = Uns.allocate(msz, throwOOME);
return address != 0L ? new Table(address, hashTableSize) : null;
}
private Table(long address, int hashTableSize) {
this.address = address;
this.mask = hashTableSize - 1;
clear();
}
void clear() {
// It's important to initialize the hash table memory.
// (uninitialized memory will cause problems - endless loops, JVM crashes, damaged data, etc)
Uns.setMemory(address, 0L, HashTableUtil.NON_MEMORY_POOL_BUCKET_ENTRY_LEN * size(), (byte) 0);
}
void release() {
Uns.free(address);
released = true;
}
protected void finalize() throws Throwable {
if (!released) {
Uns.free(address);
}
super.finalize();
}
long getFirst(long hash) {
return Uns.getLong(address, bucketOffset(hash));
}
void setFirst(long hash, long hashEntryAdr) {
Uns.putLong(address, bucketOffset(hash), hashEntryAdr);
}
long bucketOffset(long hash) {
return bucketIndexForHash(hash) * HashTableUtil.NON_MEMORY_POOL_BUCKET_ENTRY_LEN;
}
private int bucketIndexForHash(long hash) {
return (int) (hash & mask);
}
void removeLink(long hash, long hashEntryAdr, long prevEntryAdr) {
long next = NonMemoryPoolHashEntries.getNext(hashEntryAdr);
removeLinkInternal(hash, hashEntryAdr, prevEntryAdr, next);
}
void replaceSentinelLink(long hash, long hashEntryAdr, long prevEntryAdr, long newHashEntryAdr) {
NonMemoryPoolHashEntries.setNext(newHashEntryAdr, NonMemoryPoolHashEntries.getNext(hashEntryAdr));
removeLinkInternal(hash, hashEntryAdr, prevEntryAdr, newHashEntryAdr);
}
private void removeLinkInternal(long hash, long hashEntryAdr, long prevEntryAdr, long next) {
long head = getFirst(hash);
if (head == hashEntryAdr) {
setFirst(hash, next);
} else if (prevEntryAdr != 0L) {
if (prevEntryAdr == -1L) {
for (long adr = head;
adr != 0L;
prevEntryAdr = adr, adr = NonMemoryPoolHashEntries.getNext(adr)) {
if (adr == hashEntryAdr) {
break;
}
}
}
NonMemoryPoolHashEntries.setNext(prevEntryAdr, next);
}
}
void addAsHead(long hash, long hashEntryAdr) {
long head = getFirst(hash);
NonMemoryPoolHashEntries.setNext(hashEntryAdr, head);
setFirst(hash, hashEntryAdr);
}
int size() {
return mask + 1;
}
void updateBucketHistogram(EstimatedHistogram h) {
for (int i = 0; i < size(); i++) {
int len = 0;
for (long adr = getFirst(i); adr != 0L; adr = NonMemoryPoolHashEntries.getNext(adr)) {
len++;
}
h.add(len + 1);
}
}
}
private void removeInternal(long hashEntryAdr, long prevEntryAdr, long hash) {
table.removeLink(hash, hashEntryAdr, prevEntryAdr);
}
private void add(long hashEntryAdr, long hash) {
table.addAsHead(hash, hashEntryAdr);
}
@Override
public String toString() {
return String.valueOf(size);
}
}
================================================
FILE: src/main/java/com/oath/halodb/SegmentStats.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import com.google.common.base.MoreObjects;
class SegmentStats {
private final long noOfEntries;
private final long numberOfChunks;
private final long numberOfSlots;
private final long freeListSize;
public SegmentStats(long noOfEntries, long numberOfChunks, long numberOfSlots, long freeListSize) {
this.noOfEntries = noOfEntries;
this.numberOfChunks = numberOfChunks;
this.numberOfSlots = numberOfSlots;
this.freeListSize = freeListSize;
}
@Override
public String toString() {
MoreObjects.ToStringHelper helper =
MoreObjects.toStringHelper("").add("noOfEntries", this.noOfEntries);
// all these values will be -1 for non-memory pool, hence ignore.
if (numberOfChunks != -1) {
helper.add("numberOfChunks", numberOfChunks);
}
if (numberOfSlots != -1) {
helper.add("numberOfSlots", numberOfSlots);
}
if (freeListSize != -1) {
helper.add("freeListSize", freeListSize);
}
return helper.toString();
}
@Override
public boolean equals(Object obj) {
if (obj == this)
return true;
if (!(obj instanceof SegmentStats))
return false;
SegmentStats that = (SegmentStats) obj;
return that.noOfEntries == noOfEntries
&& that.numberOfChunks == numberOfChunks
&& that.numberOfSlots == numberOfSlots
&& that.freeListSize == freeListSize;
}
@Override
public int hashCode() {
int result = 1;
result = 31 * result + Long.hashCode(noOfEntries);
result = 31 * result + Long.hashCode(numberOfChunks);
result = 31 * result + Long.hashCode(numberOfSlots);
result = 31 * result + Long.hashCode(freeListSize);
return result;
}
}
================================================
FILE: src/main/java/com/oath/halodb/SegmentWithMemoryPool.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.primitives.Ints;
import com.oath.halodb.histo.EstimatedHistogram;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
class SegmentWithMemoryPool extends Segment {
private static final Logger logger = LoggerFactory.getLogger(SegmentWithMemoryPool.class);
// maximum hash table size
private static final int MAX_TABLE_SIZE = 1 << 30;
private long hitCount = 0;
private long size = 0;
private long missCount = 0;
private long putAddCount = 0;
private long putReplaceCount = 0;
private long removeCount = 0;
private long threshold = 0;
private final float loadFactor;
private long rehashes = 0;
private final List chunks;
private byte currentChunkIndex = -1;
private final int chunkSize;
private final MemoryPoolAddress emptyAddress = new MemoryPoolAddress((byte) -1, -1);
private MemoryPoolAddress freeListHead = emptyAddress;
private long freeListSize = 0;
private final int fixedSlotSize;
private final HashTableValueSerializer valueSerializer;
private Table table;
private final ByteBuffer oldValueBuffer = ByteBuffer.allocate(fixedValueLength);
private final ByteBuffer newValueBuffer = ByteBuffer.allocate(fixedValueLength);
private final HashAlgorithm hashAlgorithm;
SegmentWithMemoryPool(OffHeapHashTableBuilder builder) {
super(builder.getValueSerializer(), builder.getFixedValueSize(), builder.getFixedKeySize(),
builder.getHasher());
this.chunks = new ArrayList<>();
this.chunkSize = builder.getMemoryPoolChunkSize();
this.valueSerializer = builder.getValueSerializer();
this.fixedSlotSize = MemoryPoolHashEntries.HEADER_SIZE + fixedKeyLength + fixedValueLength;
this.hashAlgorithm = builder.getHashAlgorighm();
int hts = builder.getHashTableSize();
if (hts <= 0) {
hts = 8192;
}
if (hts < 256) {
hts = 256;
}
int msz = Ints.checkedCast(HashTableUtil.roundUpToPowerOf2(hts, MAX_TABLE_SIZE));
table = Table.create(msz);
if (table == null) {
throw new RuntimeException("unable to allocate off-heap memory for segment");
}
float lf = builder.getLoadFactor();
if (lf <= .0d) {
lf = .75f;
}
this.loadFactor = lf;
threshold = (long) ((double) table.size() * loadFactor);
}
@Override
public V getEntry(KeyBuffer key) {
boolean wasFirst = lock();
try {
for (MemoryPoolAddress address = table.getFirst(key.hash());
address.chunkIndex >= 0;
address = getNext(address)) {
MemoryPoolChunk chunk = chunks.get(address.chunkIndex);
if (chunk.compareKey(address.chunkOffset, key.buffer)) {
hitCount++;
return valueSerializer.deserialize(chunk.readOnlyValueByteBuffer(address.chunkOffset));
}
}
missCount++;
return null;
} finally {
unlock(wasFirst);
}
}
@Override
public boolean containsEntry(KeyBuffer key) {
boolean wasFirst = lock();
try {
for (MemoryPoolAddress address = table.getFirst(key.hash());
address.chunkIndex >= 0;
address = getNext(address)) {
MemoryPoolChunk chunk = chunks.get(address.chunkIndex);
if (chunk.compareKey(address.chunkOffset, key.buffer)) {
hitCount++;
return true;
}
}
missCount++;
return false;
} finally {
unlock(wasFirst);
}
}
@Override
boolean putEntry(byte[] key, V value, long hash, boolean putIfAbsent, V oldValue) {
boolean wasFirst = lock();
try {
if (oldValue != null) {
oldValueBuffer.clear();
valueSerializer.serialize(oldValue, oldValueBuffer);
}
newValueBuffer.clear();
valueSerializer.serialize(value, newValueBuffer);
MemoryPoolAddress first = table.getFirst(hash);
for (MemoryPoolAddress address = first; address.chunkIndex >= 0; address = getNext(address)) {
MemoryPoolChunk chunk = chunks.get(address.chunkIndex);
if (chunk.compareKey(address.chunkOffset, key)) {
// key is already present in the segment.
// putIfAbsent is true, but key is already present, return.
if (putIfAbsent) {
return false;
}
// code for replace() operation
if (oldValue != null) {
if (!chunk.compareValue(address.chunkOffset, oldValueBuffer.array())) {
return false;
}
}
// replace value with the new one.
chunk.setValue(newValueBuffer.array(), address.chunkOffset);
putReplaceCount++;
return true;
}
}
if (oldValue != null) {
// key is not present but old value is not null.
// we consider this as a mismatch and return.
return false;
}
if (size >= threshold) {
rehash();
first = table.getFirst(hash);
}
// key is not present in the segment, we need to add a new entry.
MemoryPoolAddress nextSlot = writeToFreeSlot(key, newValueBuffer.array(), first);
table.addAsHead(hash, nextSlot);
size++;
putAddCount++;
} finally {
unlock(wasFirst);
}
return true;
}
@Override
public boolean removeEntry(KeyBuffer key) {
boolean wasFirst = lock();
try {
MemoryPoolAddress previous = null;
for (MemoryPoolAddress address = table.getFirst(key.hash());
address.chunkIndex >= 0;
previous = address, address = getNext(address)) {
MemoryPoolChunk chunk = chunks.get(address.chunkIndex);
if (chunk.compareKey(address.chunkOffset, key.buffer)) {
removeInternal(address, previous, key.hash());
removeCount++;
size--;
return true;
}
}
return false;
} finally {
unlock(wasFirst);
}
}
private MemoryPoolAddress getNext(MemoryPoolAddress address) {
if (address.chunkIndex < 0 || address.chunkIndex >= chunks.size()) {
throw new IllegalArgumentException("Invalid chunk index " + address.chunkIndex + ". Chunk size " + chunks.size());
}
MemoryPoolChunk chunk = chunks.get(address.chunkIndex);
return chunk.getNextAddress(address.chunkOffset);
}
private MemoryPoolAddress writeToFreeSlot(byte[] key, byte[] value, MemoryPoolAddress nextAddress) {
if (!freeListHead.equals(emptyAddress)) {
// write to the head of the free list.
MemoryPoolAddress temp = freeListHead;
freeListHead = chunks.get(freeListHead.chunkIndex).getNextAddress(freeListHead.chunkOffset);
chunks.get(temp.chunkIndex).fillSlot(temp.chunkOffset, key, value, nextAddress);
--freeListSize;
return temp;
}
if (currentChunkIndex == -1 || chunks.get(currentChunkIndex).remaining() < fixedSlotSize) {
if (chunks.size() > Byte.MAX_VALUE) {
logger.error("No more memory left. Each segment can have at most {} chunks.", Byte.MAX_VALUE + 1);
throw new OutOfMemoryError("Each segment can have at most " + (Byte.MAX_VALUE + 1) + " chunks.");
}
// There is no chunk allocated for this segment or the current chunk being written to has no space left.
// allocate an new one.
chunks.add(MemoryPoolChunk.create(chunkSize, fixedKeyLength, fixedValueLength));
++currentChunkIndex;
}
MemoryPoolChunk currentWriteChunk = chunks.get(currentChunkIndex);
MemoryPoolAddress slotAddress = new MemoryPoolAddress(currentChunkIndex, currentWriteChunk.getWriteOffset());
currentWriteChunk.fillNextSlot(key, value, nextAddress);
return slotAddress;
}
private void removeInternal(MemoryPoolAddress address, MemoryPoolAddress previous, long hash) {
MemoryPoolAddress next = chunks.get(address.chunkIndex).getNextAddress(address.chunkOffset);
if (table.getFirst(hash).equals(address)) {
table.addAsHead(hash, next);
} else if (previous == null) {
//this should never happen.
throw new IllegalArgumentException("Removing entry which is not head but with previous null");
} else {
chunks.get(previous.chunkIndex).setNextAddress(previous.chunkOffset, next);
}
chunks.get(address.chunkIndex).setNextAddress(address.chunkOffset, freeListHead);
freeListHead = address;
++freeListSize;
}
private void rehash() {
long start = System.currentTimeMillis();
Table currentTable = table;
int tableSize = currentTable.size();
if (tableSize > MAX_TABLE_SIZE) {
return;
}
Table newTable = Table.create(tableSize * 2);
Hasher hasher = Hasher.create(hashAlgorithm);
MemoryPoolAddress next;
for (int i = 0; i < tableSize; i++) {
for (MemoryPoolAddress address = table.getFirst(i); address.chunkIndex >= 0; address = next) {
long hash = chunks.get(address.chunkIndex).computeHash(address.chunkOffset, hasher);
next = getNext(address);
MemoryPoolAddress first = newTable.getFirst(hash);
newTable.addAsHead(hash, address);
chunks.get(address.chunkIndex).setNextAddress(address.chunkOffset, first);
}
}
threshold = (long) ((float) newTable.size() * loadFactor);
table.release();
table = newTable;
rehashes++;
logger.info("Completed rehashing segment in {} ms.", (System.currentTimeMillis() - start));
}
@Override
long size() {
return size;
}
@Override
void release() {
boolean wasFirst = lock();
try {
chunks.forEach(MemoryPoolChunk::destroy);
chunks.clear();
currentChunkIndex = -1;
size = 0;
table.release();
} finally {
unlock(wasFirst);
}
}
@Override
void clear() {
boolean wasFirst = lock();
try {
chunks.forEach(MemoryPoolChunk::destroy);
chunks.clear();
currentChunkIndex = -1;
size = 0;
table.clear();
} finally {
unlock(wasFirst);
}
}
@Override
long hitCount() {
return hitCount;
}
@Override
long missCount() {
return missCount;
}
@Override
long putAddCount() {
return putAddCount;
}
@Override
long putReplaceCount() {
return putReplaceCount;
}
@Override
long removeCount() {
return removeCount;
}
@Override
void resetStatistics() {
rehashes = 0L;
hitCount = 0L;
missCount = 0L;
putAddCount = 0L;
putReplaceCount = 0L;
removeCount = 0L;
}
@Override
long numberOfChunks() {
return chunks.size();
}
@Override
long numberOfSlots() {
return chunks.size() * chunkSize / fixedSlotSize;
}
@Override
long freeListSize() {
return freeListSize;
}
@Override
long rehashes() {
return rehashes;
}
@Override
float loadFactor() {
return loadFactor;
}
@Override
int hashTableSize() {
return table.size();
}
@Override
void updateBucketHistogram(EstimatedHistogram hist) {
boolean wasFirst = lock();
try {
table.updateBucketHistogram(hist, chunks);
} finally {
unlock(wasFirst);
}
}
static final class Table {
final int mask;
final long address;
private boolean released;
static Table create(int hashTableSize) {
int msz = Ints.checkedCast(HashTableUtil.MEMORY_POOL_BUCKET_ENTRY_LEN * hashTableSize);
long address = Uns.allocate(msz, true);
return address != 0L ? new Table(address, hashTableSize) : null;
}
private Table(long address, int hashTableSize) {
this.address = address;
this.mask = hashTableSize - 1;
clear();
}
void clear() {
Uns.setMemory(address, 0L, HashTableUtil.MEMORY_POOL_BUCKET_ENTRY_LEN * size(), (byte) -1);
}
void release() {
Uns.free(address);
released = true;
}
protected void finalize() throws Throwable {
if (!released) {
Uns.free(address);
}
super.finalize();
}
MemoryPoolAddress getFirst(long hash) {
long bOffset = address + bucketOffset(hash);
byte chunkIndex = Uns.getByte(bOffset, 0);
int chunkOffset = Uns.getInt(bOffset, 1);
return new MemoryPoolAddress(chunkIndex, chunkOffset);
}
void addAsHead(long hash, MemoryPoolAddress entryAddress) {
long bOffset = address + bucketOffset(hash);
Uns.putByte(bOffset, 0, entryAddress.chunkIndex);
Uns.putInt(bOffset, 1, entryAddress.chunkOffset);
}
long bucketOffset(long hash) {
return bucketIndexForHash(hash) * HashTableUtil.MEMORY_POOL_BUCKET_ENTRY_LEN;
}
private int bucketIndexForHash(long hash) {
return (int) (hash & mask);
}
int size() {
return mask + 1;
}
void updateBucketHistogram(EstimatedHistogram h, final List chunks) {
for (int i = 0; i < size(); i++) {
int len = 0;
for (MemoryPoolAddress adr = getFirst(i); adr.chunkIndex >= 0;
adr = chunks.get(adr.chunkIndex).getNextAddress(adr.chunkOffset)) {
len++;
}
h.add(len + 1);
}
}
}
@VisibleForTesting
MemoryPoolAddress getFreeListHead() {
return freeListHead;
}
@VisibleForTesting
int getChunkWriteOffset(int index) {
return chunks.get(index).getWriteOffset();
}
}
================================================
FILE: src/main/java/com/oath/halodb/TombstoneEntry.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import java.nio.ByteBuffer;
import java.util.zip.CRC32;
class TombstoneEntry {
//TODO: test.
/**
* crc - 4 byte
* version - 1 byte
* Key size - 1 byte
* Sequence number - 8 byte
*/
static final int TOMBSTONE_ENTRY_HEADER_SIZE = 4 + 1 + 1 + 8;
static final int CHECKSUM_SIZE = 4;
static final int CHECKSUM_OFFSET = 0;
static final int VERSION_OFFSET = 4;
static final int SEQUENCE_NUMBER_OFFSET = 5;
static final int KEY_SIZE_OFFSET = 13;
private final byte[] key;
private final long sequenceNumber;
private final long checkSum;
private final int version;
TombstoneEntry(byte[] key, long sequenceNumber, long checkSum, int version) {
this.key = key;
this.sequenceNumber = sequenceNumber;
this.checkSum = checkSum;
this.version = version;
}
byte[] getKey() {
return key;
}
long getSequenceNumber() {
return sequenceNumber;
}
int getVersion() {
return version;
}
long getCheckSum() {
return checkSum;
}
int size() {
return TOMBSTONE_ENTRY_HEADER_SIZE + key.length;
}
ByteBuffer[] serialize() {
byte keySize = (byte)key.length;
ByteBuffer header = ByteBuffer.allocate(TOMBSTONE_ENTRY_HEADER_SIZE);
header.put(VERSION_OFFSET, (byte)version);
header.putLong(SEQUENCE_NUMBER_OFFSET, sequenceNumber);
header.put(KEY_SIZE_OFFSET, keySize);
long crc32 = computeCheckSum(header.array());
header.putInt(CHECKSUM_OFFSET, Utils.toSignedIntFromLong(crc32));
return new ByteBuffer[] {header, ByteBuffer.wrap(key)};
}
static TombstoneEntry deserialize(ByteBuffer buffer) {
long crc32 = Utils.toUnsignedIntFromInt(buffer.getInt());
int version = Utils.toUnsignedByte(buffer.get());
long sequenceNumber = buffer.getLong();
int keySize = (int)buffer.get();
byte[] key = new byte[keySize];
buffer.get(key);
return new TombstoneEntry(key, sequenceNumber, crc32, version);
}
// returns null if a corrupted entry is detected.
static TombstoneEntry deserializeIfNotCorrupted(ByteBuffer buffer) {
if (buffer.remaining() < TOMBSTONE_ENTRY_HEADER_SIZE) {
return null;
}
long crc32 = Utils.toUnsignedIntFromInt(buffer.getInt());
int version = Utils.toUnsignedByte(buffer.get());
long sequenceNumber = buffer.getLong();
int keySize = (int)buffer.get();
if (sequenceNumber < 0 || keySize <= 0 || version < 0 || version > 255 || buffer.remaining() < keySize)
return null;
byte[] key = new byte[keySize];
buffer.get(key);
TombstoneEntry entry = new TombstoneEntry(key, sequenceNumber, crc32, version);
if (entry.computeCheckSum() != entry.checkSum) {
return null;
}
return entry;
}
private long computeCheckSum(byte[] header) {
CRC32 crc32 = new CRC32();
crc32.update(header, CHECKSUM_OFFSET + CHECKSUM_SIZE, TOMBSTONE_ENTRY_HEADER_SIZE - CHECKSUM_SIZE);
crc32.update(key);
return crc32.getValue();
}
long computeCheckSum() {
ByteBuffer header = ByteBuffer.allocate(TOMBSTONE_ENTRY_HEADER_SIZE);
header.put(VERSION_OFFSET, (byte)version);
header.putLong(SEQUENCE_NUMBER_OFFSET, sequenceNumber);
header.put(KEY_SIZE_OFFSET, (byte)key.length);
return computeCheckSum(header.array());
}
}
================================================
FILE: src/main/java/com/oath/halodb/TombstoneFile.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Iterator;
import java.util.Objects;
import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
class TombstoneFile {
private static final Logger logger = LoggerFactory.getLogger(TombstoneFile.class);
private final File backingFile;
private FileChannel channel;
private final DBDirectory dbDirectory;
private final HaloDBOptions options;
private long unFlushedData = 0;
private long writeOffset = 0;
static final String TOMBSTONE_FILE_NAME = ".tombstone";
private static final String nullMessage = "Tombstone entry cannot be null";
static TombstoneFile create(DBDirectory dbDirectory, int fileId, HaloDBOptions options) throws IOException {
File file = getTombstoneFile(dbDirectory, fileId);
while (!file.createNewFile()) {
// file already exists try another one.
fileId++;
file = getTombstoneFile(dbDirectory, fileId);
}
TombstoneFile tombstoneFile = new TombstoneFile(file, options, dbDirectory);
tombstoneFile.open();
return tombstoneFile;
}
TombstoneFile(File backingFile, HaloDBOptions options, DBDirectory dbDirectory) {
this.backingFile = backingFile;
this.options = options;
this.dbDirectory = dbDirectory;
}
void open() throws IOException {
channel = new RandomAccessFile(backingFile, "rw").getChannel();
}
void close() throws IOException {
if (channel != null) {
channel.close();
}
}
void delete() throws IOException {
close();
if (backingFile != null) {
backingFile.delete();
}
}
void write(TombstoneEntry entry) throws IOException {
Objects.requireNonNull(entry, nullMessage);
ByteBuffer[] contents = entry.serialize();
long toWrite = 0;
for (ByteBuffer buffer : contents) {
toWrite += buffer.remaining();
}
long written = 0;
while (written < toWrite) {
written += channel.write(contents);
}
writeOffset += written;
unFlushedData += written;
if (options.isSyncWrite() || (options.getFlushDataSizeBytes() != -1 && unFlushedData > options.getFlushDataSizeBytes())) {
flushToDisk();
unFlushedData = 0;
}
}
long getWriteOffset() {
return writeOffset;
}
void flushToDisk() throws IOException {
if (channel != null && channel.isOpen())
channel.force(true);
}
/**
* Copies to a temp file those entries whose computed checksum matches the stored one and then
* atomically rename the temp file to the current file.
* Records in the file which occur after a corrupted record are discarded.
* Current file is deleted after copy.
* This method is called if we detect an unclean shutdown.
*/
TombstoneFile repairFile(DBDirectory dbDirectory) throws IOException {
TombstoneFile repairFile = createRepairFile();
logger.info("Repairing tombstone file {}. Records with the correct checksum will be copied to {}", getName(), repairFile.getName());
TombstoneFileIterator iterator = newIteratorWithCheckForDataCorruption();
int count = 0;
while (iterator.hasNext()) {
TombstoneEntry entry = iterator.next();
if (entry == null) {
logger.info("Found a corrupted entry in tombstone file {} after copying {} entries.", getName(), count);
break;
}
count++;
repairFile.write(entry);
}
logger.info("Recovered {} records from file {} with size {}. Size after repair {}.", count, getName(), getSize(), repairFile.getSize());
repairFile.flushToDisk();
Files.move(repairFile.getPath(), getPath(), REPLACE_EXISTING, ATOMIC_MOVE);
dbDirectory.syncMetaData();
repairFile.close();
close();
open();
return this;
}
private TombstoneFile createRepairFile() throws IOException {
File repairFile = dbDirectory.getPath().resolve(getName()+".repair").toFile();
while (!repairFile.createNewFile()) {
logger.info("Repair file {} already exists, probably from a previous repair which failed. Deleting a trying again", repairFile.getName());
repairFile.delete();
}
TombstoneFile tombstoneFile = new TombstoneFile(repairFile, options, dbDirectory);
tombstoneFile.open();
return tombstoneFile;
}
String getName() {
return backingFile.getName();
}
private Path getPath() {
return backingFile.toPath();
}
private long getSize() {
return backingFile.length();
}
TombstoneFile.TombstoneFileIterator newIterator() throws IOException {
return new TombstoneFile.TombstoneFileIterator(false);
}
// Returns null when it finds a corrupted entry.
TombstoneFile.TombstoneFileIterator newIteratorWithCheckForDataCorruption() throws IOException {
return new TombstoneFile.TombstoneFileIterator(true);
}
private static File getTombstoneFile(DBDirectory dbDirectory, int fileId) {
return dbDirectory.getPath().resolve(fileId + TOMBSTONE_FILE_NAME).toFile();
}
class TombstoneFileIterator implements Iterator {
private final ByteBuffer buffer;
private final boolean discardCorruptedRecords;
TombstoneFileIterator(boolean discardCorruptedRecords) throws IOException {
buffer = channel.map(FileChannel.MapMode.READ_ONLY, 0, channel.size());
this.discardCorruptedRecords = discardCorruptedRecords;
}
@Override
public boolean hasNext() {
return buffer.hasRemaining();
}
@Override
public TombstoneEntry next() {
if (hasNext()) {
if (discardCorruptedRecords)
return TombstoneEntry.deserializeIfNotCorrupted(buffer);
return TombstoneEntry.deserialize(buffer);
}
return null;
}
}
}
================================================
FILE: src/main/java/com/oath/halodb/Uns.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import sun.misc.Unsafe;
import java.io.IOException;
import java.lang.reflect.Field;
import java.nio.Buffer;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.Map;
import java.util.StringTokenizer;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
final class Uns {
private static final Logger LOGGER = LoggerFactory.getLogger(Uns.class);
private static final Unsafe unsafe;
private static final NativeMemoryAllocator allocator;
private static final boolean __DEBUG_OFF_HEAP_MEMORY_ACCESS = Boolean.parseBoolean(System.getProperty(OffHeapHashTableBuilder.SYSTEM_PROPERTY_PREFIX + "debugOffHeapAccess", "false"));
private static final String __ALLOCATOR = System.getProperty(OffHeapHashTableBuilder.SYSTEM_PROPERTY_PREFIX + "allocator");
//
// #ifdef __DEBUG_OFF_HEAP_MEMORY_ACCESS
//
private static final ConcurrentMap ohDebug = __DEBUG_OFF_HEAP_MEMORY_ACCESS ? new ConcurrentHashMap(16384) : null;
private static final Map ohFreeDebug = __DEBUG_OFF_HEAP_MEMORY_ACCESS ? new ConcurrentHashMap(16384) : null;
private static final class AllocInfo {
final long size;
final Throwable trace;
AllocInfo(Long size, Throwable trace) {
this.size = size;
this.trace = trace;
}
}
static void clearUnsDebugForTest() {
if (__DEBUG_OFF_HEAP_MEMORY_ACCESS) {
try {
if (!ohDebug.isEmpty()) {
for (Map.Entry addrSize : ohDebug.entrySet()) {
System.err.printf(" still allocated: address=%d, size=%d%n", addrSize.getKey(), addrSize.getValue().size);
addrSize.getValue().trace.printStackTrace();
}
throw new RuntimeException("Not all allocated memory has been freed!");
}
} finally {
ohDebug.clear();
ohFreeDebug.clear();
}
}
}
private static void freed(long address) {
if (__DEBUG_OFF_HEAP_MEMORY_ACCESS) {
AllocInfo allocInfo = ohDebug.remove(address);
if (allocInfo == null) {
Throwable freedAt = ohFreeDebug.get(address);
throw new IllegalStateException("Free of unallocated region " + address, freedAt);
}
ohFreeDebug.put(address, new Exception("free backtrace - t=" + System.nanoTime()));
}
}
private static void allocated(long address, long bytes) {
if (__DEBUG_OFF_HEAP_MEMORY_ACCESS) {
AllocInfo allocatedLen =
ohDebug.putIfAbsent(address, new AllocInfo(bytes, new Exception("Thread: " + Thread.currentThread())));
if (allocatedLen != null) {
throw new Error("Oops - allocate() got duplicate address");
}
ohFreeDebug.remove(address);
}
}
private static void validate(long address, long offset, long len) {
if (__DEBUG_OFF_HEAP_MEMORY_ACCESS) {
if (address == 0L) {
throw new NullPointerException();
}
AllocInfo allocInfo = ohDebug.get(address);
if (allocInfo == null) {
Throwable freedAt = ohFreeDebug.get(address);
throw new IllegalStateException("Access to unallocated region " + address + " - t=" + System.nanoTime(), freedAt);
}
if (offset < 0L) {
throw new IllegalArgumentException("Negative offset");
}
if (len < 0L) {
throw new IllegalArgumentException("Negative length");
}
if (offset + len > allocInfo.size) {
throw new IllegalArgumentException("Access outside allocated region");
}
}
}
//
// #endif
//
private static final UnsExt ext;
static {
try {
Field field = Unsafe.class.getDeclaredField("theUnsafe");
field.setAccessible(true);
unsafe = (Unsafe) field.get(null);
if (unsafe.addressSize() > 8) {
throw new RuntimeException("Address size " + unsafe.addressSize() + " not supported yet (max 8 bytes)");
}
String javaVersion = System.getProperty("java.version");
if (javaVersion.indexOf('-') != -1) {
javaVersion = javaVersion.substring(0, javaVersion.indexOf('-'));
}
StringTokenizer st = new StringTokenizer(javaVersion, ".");
int major = Integer.parseInt(st.nextToken());
int minor = st.hasMoreTokens() ? Integer.parseInt(st.nextToken()) : 0;
UnsExt e;
if (major > 1 || minor >= 8) {
try {
// use new Java8 methods in sun.misc.Unsafe
ext = new UnsExt8(unsafe);
LOGGER.info("OHC using Java8 Unsafe API");
} catch (VirtualMachineError ex) {
throw ex;
}
} else {
throw new RuntimeException("HaloDB requires java version >= 1.8");
}
if (__DEBUG_OFF_HEAP_MEMORY_ACCESS)
LOGGER.warn("Degraded performance due to off-heap memory allocations and access guarded by debug code enabled via system property " + OffHeapHashTableBuilder.SYSTEM_PROPERTY_PREFIX + "debugOffHeapAccess=true");
NativeMemoryAllocator alloc;
String allocType = __ALLOCATOR != null ? __ALLOCATOR : "jna";
switch (allocType) {
case "unsafe":
alloc = new UnsafeAllocator();
LOGGER.info("OHC using sun.misc.Unsafe memory allocation");
break;
case "jna":
default:
alloc = new JNANativeAllocator();
LOGGER.info("OHC using JNA OS native malloc/free");
}
allocator = alloc;
} catch (Exception e) {
throw new AssertionError(e);
}
}
private Uns() {
}
static long getLongFromByteArray(byte[] array, int offset) {
if (offset < 0 || offset + 8 > array.length)
throw new ArrayIndexOutOfBoundsException();
return unsafe.getLong(array, (long) Unsafe.ARRAY_BYTE_BASE_OFFSET + offset);
}
static int getIntFromByteArray(byte[] array, int offset) {
if (offset < 0 || offset + 4 > array.length) {
throw new ArrayIndexOutOfBoundsException();
}
return unsafe.getInt(array, (long) Unsafe.ARRAY_BYTE_BASE_OFFSET + offset);
}
static short getShortFromByteArray(byte[] array, int offset) {
if (offset < 0 || offset + 2 > array.length) {
throw new ArrayIndexOutOfBoundsException();
}
return unsafe.getShort(array, (long) Unsafe.ARRAY_BYTE_BASE_OFFSET + offset);
}
static long getAndPutLong(long address, long offset, long value) {
validate(address, offset, 8L);
return ext.getAndPutLong(address, offset, value);
}
static void putLong(long address, long offset, long value) {
validate(address, offset, 8L);
unsafe.putLong(null, address + offset, value);
}
static long getLong(long address, long offset) {
validate(address, offset, 8L);
return unsafe.getLong(null, address + offset);
}
static void putInt(long address, long offset, int value) {
validate(address, offset, 4L);
unsafe.putInt(null, address + offset, value);
}
static int getInt(long address, long offset) {
validate(address, offset, 4L);
return unsafe.getInt(null, address + offset);
}
static void putShort(long address, long offset, short value) {
validate(address, offset, 2L);
unsafe.putShort(null, address + offset, value);
}
static short getShort(long address, long offset) {
validate(address, offset, 2L);
return unsafe.getShort(null, address + offset);
}
static void putByte(long address, long offset, byte value) {
validate(address, offset, 1L);
unsafe.putByte(null, address + offset, value);
}
static byte getByte(long address, long offset) {
validate(address, offset, 1L);
return unsafe.getByte(null, address + offset);
}
static boolean decrement(long address, long offset) {
validate(address, offset, 4L);
long v = ext.getAndAddInt(address, offset, -1);
return v == 1;
}
static void increment(long address, long offset) {
validate(address, offset, 4L);
ext.getAndAddInt(address, offset, 1);
}
static void copyMemory(byte[] arr, int off, long address, long offset, long len) {
validate(address, offset, len);
unsafe.copyMemory(arr, Unsafe.ARRAY_BYTE_BASE_OFFSET + off, null, address + offset, len);
}
static void copyMemory(long address, long offset, byte[] arr, int off, long len) {
validate(address, offset, len);
unsafe.copyMemory(null, address + offset, arr, Unsafe.ARRAY_BYTE_BASE_OFFSET + off, len);
}
static void copyMemory(long src, long srcOffset, long dst, long dstOffset, long len) {
validate(src, srcOffset, len);
validate(dst, dstOffset, len);
unsafe.copyMemory(null, src + srcOffset, null, dst + dstOffset, len);
}
static void setMemory(long address, long offset, long len, byte val) {
validate(address, offset, len);
unsafe.setMemory(address + offset, len, val);
}
static boolean memoryCompare(long adr1, long off1, long adr2, long off2, long len) {
if (adr1 == 0L) {
return false;
}
if (adr1 == adr2) {
assert off1 == off2;
return true;
}
for (; len >= 8; len -= 8, off1 += 8, off2 += 8) {
if (Uns.getLong(adr1, off1) != Uns.getLong(adr2, off2)) {
return false;
}
}
for (; len >= 4; len -= 4, off1 += 4, off2 += 4) {
if (Uns.getInt(adr1, off1) != Uns.getInt(adr2, off2)) {
return false;
}
}
for (; len >= 2; len -= 2, off1 += 2, off2 += 2) {
if (Uns.getShort(adr1, off1) != Uns.getShort(adr2, off2)) {
return false;
}
}
for (; len > 0; len--, off1++, off2++) {
if (Uns.getByte(adr1, off1) != Uns.getByte(adr2, off2)) {
return false;
}
}
return true;
}
static long crc32(long address, long offset, long len) {
validate(address, offset, len);
return ext.crc32(address, offset, len);
}
static long getTotalAllocated() {
return allocator.getTotalAllocated();
}
static long allocate(long bytes) {
return allocate(bytes, false);
}
static long allocate(long bytes, boolean throwOOME) {
long address = allocator.allocate(bytes);
if (address != 0L) {
allocated(address, bytes);
} else if (throwOOME) {
throw new OutOfMemoryError("unable to allocate " + bytes + " in off-heap");
}
return address;
}
static long allocateIOException(long bytes) throws IOException {
return allocateIOException(bytes, false);
}
static long allocateIOException(long bytes, boolean throwOOME) throws IOException {
long address = allocate(bytes, throwOOME);
if (address == 0L) {
throw new IOException("unable to allocate " + bytes + " in off-heap");
}
return address;
}
static void free(long address) {
if (address == 0L) {
return;
}
freed(address);
allocator.free(address);
}
private static final Class> DIRECT_BYTE_BUFFER_CLASS;
private static final Class> DIRECT_BYTE_BUFFER_CLASS_R;
private static final long DIRECT_BYTE_BUFFER_ADDRESS_OFFSET;
private static final long DIRECT_BYTE_BUFFER_CAPACITY_OFFSET;
private static final long DIRECT_BYTE_BUFFER_LIMIT_OFFSET;
static {
try {
ByteBuffer directBuffer = ByteBuffer.allocateDirect(0);
ByteBuffer directReadOnly = directBuffer.asReadOnlyBuffer();
Class> clazz = directBuffer.getClass();
Class> clazzReadOnly = directReadOnly.getClass();
DIRECT_BYTE_BUFFER_ADDRESS_OFFSET = unsafe.objectFieldOffset(Buffer.class.getDeclaredField("address"));
DIRECT_BYTE_BUFFER_CAPACITY_OFFSET = unsafe.objectFieldOffset(Buffer.class.getDeclaredField("capacity"));
DIRECT_BYTE_BUFFER_LIMIT_OFFSET = unsafe.objectFieldOffset(Buffer.class.getDeclaredField("limit"));
DIRECT_BYTE_BUFFER_CLASS = clazz;
DIRECT_BYTE_BUFFER_CLASS_R = clazzReadOnly;
} catch (NoSuchFieldException e) {
throw new RuntimeException(e);
}
}
static ByteBuffer directBufferFor(long address, long offset, long len, boolean readOnly) {
if (len > Integer.MAX_VALUE || len < 0L) {
throw new IllegalArgumentException();
}
try {
ByteBuffer bb = (ByteBuffer) unsafe.allocateInstance(readOnly ? DIRECT_BYTE_BUFFER_CLASS_R : DIRECT_BYTE_BUFFER_CLASS);
unsafe.putLong(bb, DIRECT_BYTE_BUFFER_ADDRESS_OFFSET, address + offset);
unsafe.putInt(bb, DIRECT_BYTE_BUFFER_CAPACITY_OFFSET, (int) len);
unsafe.putInt(bb, DIRECT_BYTE_BUFFER_LIMIT_OFFSET, (int) len);
bb.order(ByteOrder.BIG_ENDIAN);
return bb;
} catch (Error e) {
throw e;
} catch (Throwable t) {
throw new RuntimeException(t);
}
}
static void invalidateDirectBuffer(ByteBuffer buffer) {
buffer.position(0);
unsafe.putInt(buffer, DIRECT_BYTE_BUFFER_CAPACITY_OFFSET, 0);
unsafe.putInt(buffer, DIRECT_BYTE_BUFFER_LIMIT_OFFSET, 0);
unsafe.putLong(buffer, DIRECT_BYTE_BUFFER_ADDRESS_OFFSET, 0L);
}
static ByteBuffer readOnlyBuffer(long hashEntryAdr, int length, long offset) {
return Uns.directBufferFor(hashEntryAdr + offset, 0, length, true);
}
static ByteBuffer buffer(long hashEntryAdr, long length, long offset) {
return Uns.directBufferFor(hashEntryAdr + offset, 0, length, false);
}
}
================================================
FILE: src/main/java/com/oath/halodb/UnsExt.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import sun.misc.Unsafe;
abstract class UnsExt {
final Unsafe unsafe;
UnsExt(Unsafe unsafe) {
this.unsafe = unsafe;
}
abstract long getAndPutLong(long address, long offset, long value);
abstract int getAndAddInt(long address, long offset, int value);
abstract long crc32(long address, long offset, long len);
}
================================================
FILE: src/main/java/com/oath/halodb/UnsExt8.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import sun.misc.Unsafe;
import java.util.zip.CRC32;
final class UnsExt8 extends UnsExt {
UnsExt8(Unsafe unsafe) {
super(unsafe);
}
long getAndPutLong(long address, long offset, long value) {
return unsafe.getAndSetLong(null, address + offset, value);
}
int getAndAddInt(long address, long offset, int value) {
return unsafe.getAndAddInt(null, address + offset, value);
}
long crc32(long address, long offset, long len) {
CRC32 crc = new CRC32();
crc.update(Uns.directBufferFor(address, offset, len, true));
long h = crc.getValue();
h |= h << 32;
return h;
}
}
================================================
FILE: src/main/java/com/oath/halodb/UnsafeAllocator.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import sun.misc.Unsafe;
import java.lang.reflect.Field;
final class UnsafeAllocator implements NativeMemoryAllocator {
static final Unsafe unsafe;
static {
try {
Field field = Unsafe.class.getDeclaredField("theUnsafe");
field.setAccessible(true);
unsafe = (Unsafe) field.get(null);
} catch (Exception e) {
throw new AssertionError(e);
}
}
public long allocate(long size) {
try {
return unsafe.allocateMemory(size);
} catch (OutOfMemoryError oom) {
return 0L;
}
}
public void free(long peer) {
unsafe.freeMemory(peer);
}
public long getTotalAllocated() {
return -1L;
}
}
================================================
FILE: src/main/java/com/oath/halodb/Utils.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
class Utils {
static long roundUpToPowerOf2(long number) {
return (number > 1) ? Long.highestOneBit((number - 1) << 1) : 1;
}
static int getValueOffset(int recordOffset, byte[] key) {
return recordOffset + Record.Header.HEADER_SIZE + key.length;
}
//TODO: probably belongs to Record.
static int getRecordSize(int keySize, int valueSize) {
return keySize + valueSize + Record.Header.HEADER_SIZE;
}
static int getValueSize(int recordSize, byte[] key) {
return recordSize - Record.Header.HEADER_SIZE - key.length;
}
static InMemoryIndexMetaData getMetaData(IndexFileEntry entry, int fileId) {
return new InMemoryIndexMetaData(fileId, Utils.getValueOffset(entry.getRecordOffset(), entry.getKey()), Utils.getValueSize(entry.getRecordSize(), entry.getKey()), entry.getSequenceNumber());
}
static long toUnsignedIntFromInt(int value) {
return value & 0xffffffffL;
}
static int toSignedIntFromLong(long value) {
return (int)(value & 0xffffffffL);
}
static int toUnsignedByte(byte value) {
return value & 0xFF;
}
}
================================================
FILE: src/main/java/com/oath/halodb/Versions.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
class Versions {
static final int CURRENT_DATA_FILE_VERSION = 0;
static final int CURRENT_INDEX_FILE_VERSION = 0;
static final int CURRENT_TOMBSTONE_FILE_VERSION = 0;
static final int CURRENT_META_FILE_VERSION = 0;
}
================================================
FILE: src/main/java/com/oath/halodb/histo/EstimatedHistogram.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb.histo;
import com.google.common.base.Objects;
import org.slf4j.Logger;
import java.util.Arrays;
import java.util.concurrent.atomic.AtomicLongArray;
public class EstimatedHistogram {
/**
* The series of values to which the counts in `buckets` correspond: 1, 2, 3, 4, 5, 6, 7, 8, 10, 12, 14, 17, 20,
* etc. Thus, a `buckets` of [0, 0, 1, 10] would mean we had seen one value of 3 and 10 values of 4.
*
* The series starts at 1 and grows by 1.2 each time (rounding and removing duplicates). It goes from 1 to around
* 36M by default (creating 90+1 buckets), which will give us timing resolution from microseconds to 36 seconds,
* with less precision as the numbers get larger.
*
* Each bucket represents values from (previous bucket offset, current offset].
*/
private final long[] bucketOffsets;
// buckets is one element longer than bucketOffsets -- the last element is values greater than the last offset
final AtomicLongArray buckets;
public EstimatedHistogram() {
this(90);
}
public EstimatedHistogram(int bucketCount) {
bucketOffsets = newOffsets(bucketCount);
buckets = new AtomicLongArray(bucketOffsets.length + 1);
}
public EstimatedHistogram(long[] offsets, long[] bucketData) {
assert bucketData.length == offsets.length + 1;
bucketOffsets = offsets;
buckets = new AtomicLongArray(bucketData);
}
private static long[] newOffsets(int size) {
long[] result = new long[size];
long last = 1;
result[0] = last;
for (int i = 1; i < size; i++) {
long next = Math.round(last * 1.2);
if (next == last) {
next++;
}
result[i] = next;
last = next;
}
return result;
}
/**
* @return the histogram values corresponding to each bucket index
*/
public long[] getBucketOffsets() {
return bucketOffsets;
}
/**
* Increments the count of the bucket closest to n, rounding UP.
*/
public void add(long n) {
int index = Arrays.binarySearch(bucketOffsets, n);
if (index < 0) {
// inexact match, take the first bucket higher than n
index = -index - 1;
}
// else exact match; we're good
buckets.incrementAndGet(index);
}
/**
* @return the count in the given bucket
*/
long get(int bucket) {
return buckets.get(bucket);
}
/**
* @param reset zero out buckets afterwards if true
* @return a long[] containing the current histogram buckets
*/
public long[] getBuckets(boolean reset) {
final int len = buckets.length();
long[] rv = new long[len];
if (reset) {
for (int i = 0; i < len; i++) {
rv[i] = buckets.getAndSet(i, 0L);
}
} else {
for (int i = 0; i < len; i++) {
rv[i] = buckets.get(i);
}
}
return rv;
}
/**
* @return the smallest value that could have been added to this histogram
*/
public long min() {
for (int i = 0; i < buckets.length(); i++) {
if (buckets.get(i) > 0) {
return i == 0 ? 0 : 1 + bucketOffsets[i - 1];
}
}
return 0;
}
/**
* @return the largest value that could have been added to this histogram. If the histogram overflowed, returns
* Long.MAX_VALUE.
*/
public long max() {
int lastBucket = buckets.length() - 1;
if (buckets.get(lastBucket) > 0) {
return Long.MAX_VALUE;
}
for (int i = lastBucket - 1; i >= 0; i--) {
if (buckets.get(i) > 0) {
return bucketOffsets[i];
}
}
return 0;
}
/**
* @return estimated value at given percentile
*/
public long percentile(double percentile) {
assert percentile >= 0 && percentile <= 1.0;
int lastBucket = buckets.length() - 1;
if (buckets.get(lastBucket) > 0) {
throw new IllegalStateException("Unable to compute when histogram overflowed");
}
long pcount = (long) Math.floor(count() * percentile);
if (pcount == 0) {
return 0;
}
long elements = 0;
for (int i = 0; i < lastBucket; i++) {
elements += buckets.get(i);
if (elements >= pcount) {
return bucketOffsets[i];
}
}
return 0;
}
/**
* @return the mean histogram value (average of bucket offsets, weighted by count)
* @throws IllegalStateException if any values were greater than the largest bucket threshold
*/
public long mean() {
int lastBucket = buckets.length() - 1;
if (buckets.get(lastBucket) > 0) {
throw new IllegalStateException("Unable to compute ceiling for max when histogram overflowed");
}
long elements = 0;
long sum = 0;
for (int i = 0; i < lastBucket; i++) {
long bCount = buckets.get(i);
elements += bCount;
sum += bCount * bucketOffsets[i];
}
return (long) Math.ceil((double) sum / elements);
}
/**
* @return the total number of non-zero values
*/
public long count() {
long sum = 0L;
for (int i = 0; i < buckets.length(); i++) {
sum += buckets.get(i);
}
return sum;
}
/**
* @return true if this histogram has overflowed -- that is, a value larger than our largest bucket could bound was
* added
*/
public boolean isOverflowed() {
return buckets.get(buckets.length() - 1) > 0;
}
/**
* log.debug() every record in the histogram
*/
public void log(Logger log) {
// only print overflow if there is any
int nameCount;
if (buckets.get(buckets.length() - 1) == 0) {
nameCount = buckets.length() - 1;
} else {
nameCount = buckets.length();
}
String[] names = new String[nameCount];
int maxNameLength = 0;
for (int i = 0; i < nameCount; i++) {
names[i] = nameOfRange(bucketOffsets, i);
maxNameLength = Math.max(maxNameLength, names[i].length());
}
// emit log records
String formatstr = "%" + maxNameLength + "s: %d";
for (int i = 0; i < nameCount; i++) {
long count = buckets.get(i);
// sort-of-hack to not print empty ranges at the start that are only used to demarcate the
// first populated range. for code clarity we don't omit this record from the maxNameLength
// calculation, and accept the unnecessary whitespace prefixes that will occasionally occur
if (i == 0 && count == 0) {
continue;
}
log.debug(String.format(formatstr, names[i], count));
}
}
public String toString() {
// only print overflow if there is any
int nameCount;
if (buckets.get(buckets.length() - 1) == 0) {
nameCount = buckets.length() - 1;
} else {
nameCount = buckets.length();
}
String[] names = new String[nameCount];
int maxNameLength = 0;
for (int i = 0; i < nameCount; i++) {
names[i] = nameOfRange(bucketOffsets, i);
maxNameLength = Math.max(maxNameLength, names[i].length());
}
StringBuilder sb = new StringBuilder();
// emit log records
String formatstr = "%" + maxNameLength + "s: %d\n";
for (int i = 0; i < nameCount; i++) {
long count = buckets.get(i);
// sort-of-hack to not print empty ranges at the start that are only used to demarcate the
// first populated range. for code clarity we don't omit this record from the maxNameLength
// calculation, and accept the unnecessary whitespace prefixes that will occasionally occur
if (i == 0 && count == 0) {
continue;
}
sb.append(String.format(formatstr, names[i], count));
}
return sb.toString();
}
private static String nameOfRange(long[] bucketOffsets, int index) {
StringBuilder sb = new StringBuilder();
appendRange(sb, bucketOffsets, index);
return sb.toString();
}
private static void appendRange(StringBuilder sb, long[] bucketOffsets, int index) {
sb.append("[");
if (index == 0) {
if (bucketOffsets[0] > 0)
// by original definition, this histogram is for values greater than zero only;
// if values of 0 or less are required, an entry of lb-1 must be inserted at the start
{
sb.append("1");
} else {
sb.append("-Inf");
}
} else {
sb.append(bucketOffsets[index - 1] + 1);
}
sb.append("..");
if (index == bucketOffsets.length) {
sb.append("Inf");
} else {
sb.append(bucketOffsets[index]);
}
sb.append("]");
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (!(o instanceof EstimatedHistogram)) {
return false;
}
EstimatedHistogram that = (EstimatedHistogram) o;
return Arrays.equals(getBucketOffsets(), that.getBucketOffsets()) &&
Arrays.equals(getBuckets(false), that.getBuckets(false));
}
@Override
public int hashCode() {
return Objects.hashCode(getBucketOffsets(), getBuckets(false));
}
}
================================================
FILE: src/test/java/com/oath/halodb/CheckOffHeapHashTable.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import com.oath.halodb.histo.EstimatedHistogram;
import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicLong;
/**
* This is a {@link OffHeapHashTable} implementation used to validate functionality of
* {@link OffHeapHashTableImpl} - this implementation is not for production use!
*/
final class CheckOffHeapHashTable implements OffHeapHashTable
{
private final HashTableValueSerializer valueSerializer;
private final CheckSegment[] maps;
private final int segmentShift;
private final long segmentMask;
private final float loadFactor;
private long putFailCount;
private final Hasher hasher;
CheckOffHeapHashTable(OffHeapHashTableBuilder builder)
{
loadFactor = builder.getLoadFactor();
hasher = Hasher.create(builder.getHashAlgorighm());
int segments = builder.getSegmentCount();
int bitNum = HashTableUtil.bitNum(segments) - 1;
this.segmentShift = 64 - bitNum;
this.segmentMask = ((long) segments - 1) << segmentShift;
maps = new CheckSegment[segments];
for (int i = 0; i < maps.length; i++)
maps[i] = new CheckSegment(builder.getHashTableSize(), builder.getLoadFactor());
valueSerializer = builder.getValueSerializer();
}
public boolean put(byte[] key, V value)
{
KeyBuffer keyBuffer = keySource(key);
byte[] data = value(value);
CheckSegment segment = segment(keyBuffer.hash());
return segment.put(keyBuffer, data, false, null);
}
public boolean addOrReplace(byte[] key, V old, V value)
{
KeyBuffer keyBuffer = keySource(key);
byte[] data = value(value);
byte[] oldData = value(old);
CheckSegment segment = segment(keyBuffer.hash());
return segment.put(keyBuffer, data, false, oldData);
}
public boolean putIfAbsent(byte[] key, V v)
{
KeyBuffer keyBuffer = keySource(key);
byte[] data = value(v);
CheckSegment segment = segment(keyBuffer.hash());
return segment.put(keyBuffer, data, true, null);
}
public boolean putIfAbsent(byte[] key, V value, long expireAt)
{
throw new UnsupportedOperationException();
}
public boolean put(byte[] key, V value, long expireAt)
{
throw new UnsupportedOperationException();
}
public boolean remove(byte[] key)
{
KeyBuffer keyBuffer = keySource(key);
CheckSegment segment = segment(keyBuffer.hash());
return segment.remove(keyBuffer);
}
public void clear()
{
for (CheckSegment map : maps)
map.clear();
}
public V get(byte[] key)
{
KeyBuffer keyBuffer = keySource(key);
CheckSegment segment = segment(keyBuffer.hash());
byte[] value = segment.get(keyBuffer);
if (value == null)
return null;
return valueSerializer.deserialize(ByteBuffer.wrap(value));
}
public boolean containsKey(byte[] key)
{
KeyBuffer keyBuffer = keySource(key);
CheckSegment segment = segment(keyBuffer.hash());
return segment.get(keyBuffer) != null;
}
public void resetStatistics()
{
for (CheckSegment map : maps)
map.resetStatistics();
putFailCount = 0;
}
public long size()
{
long r = 0;
for (CheckSegment map : maps)
r += map.size();
return r;
}
public int[] hashTableSizes()
{
// no hash table size info
return new int[maps.length];
}
public SegmentStats[] perSegmentStats() {
SegmentStats[] stats = new SegmentStats[maps.length];
for (int i = 0; i < stats.length; i++) {
CheckSegment map = maps[i];
stats[i] = new SegmentStats(map.size(), -1, -1, -1);
}
return stats;
}
public EstimatedHistogram getBucketHistogram()
{
throw new UnsupportedOperationException();
}
public int segments()
{
return maps.length;
}
public float loadFactor()
{
return loadFactor;
}
public OffHeapHashTableStats stats()
{
return new OffHeapHashTableStats(
hitCount(),
missCount(),
size(),
-1L,
putAddCount(),
putReplaceCount(),
putFailCount,
removeCount(),
perSegmentStats()
);
}
private long putAddCount()
{
long putAddCount = 0L;
for (CheckSegment map : maps)
putAddCount += map.putAddCount;
return putAddCount;
}
private long putReplaceCount()
{
long putReplaceCount = 0L;
for (CheckSegment map : maps)
putReplaceCount += map.putReplaceCount;
return putReplaceCount;
}
private long removeCount()
{
long removeCount = 0L;
for (CheckSegment map : maps)
removeCount += map.removeCount;
return removeCount;
}
private long hitCount()
{
long hitCount = 0L;
for (CheckSegment map : maps)
hitCount += map.hitCount;
return hitCount;
}
private long missCount()
{
long missCount = 0L;
for (CheckSegment map : maps)
missCount += map.missCount;
return missCount;
}
public void close()
{
clear();
}
//
//
//
private CheckSegment segment(long hash)
{
int seg = (int) ((hash & segmentMask) >>> segmentShift);
return maps[seg];
}
KeyBuffer keySource(byte[] key) {
KeyBuffer keyBuffer = new KeyBuffer(key);
return keyBuffer.finish(hasher);
}
private byte[] value(V value)
{
if (value == null) {
return null;
}
ByteBuffer buf = ByteBuffer.allocate(valueSerializer.serializedSize(value));
valueSerializer.serialize(value, buf);
return buf.array();
}
}
================================================
FILE: src/test/java/com/oath/halodb/CheckSegment.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import java.util.Arrays;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
/**
* On-heap test-only counterpart of {@link SegmentNonMemoryPool} for {@link CheckOffHeapHashTable}.
*/
final class CheckSegment {
private final Map map;
private final LinkedList lru = new LinkedList<>();
long hitCount;
long missCount;
long putAddCount;
long putReplaceCount;
long removeCount;
long evictedEntries;
public CheckSegment(int initialCapacity, float loadFactor) {
this.map = new HashMap<>(initialCapacity, loadFactor);
}
synchronized void clear()
{
map.clear();
lru.clear();
}
synchronized byte[] get(KeyBuffer keyBuffer)
{
byte[] r = map.get(keyBuffer);
if (r == null)
{
missCount++;
return null;
}
lru.remove(keyBuffer);
lru.addFirst(keyBuffer);
hitCount++;
return r;
}
synchronized boolean put(KeyBuffer keyBuffer, byte[] data, boolean ifAbsent, byte[] old)
{
byte[] existing = map.get(keyBuffer);
if (ifAbsent && existing != null)
return false;
if (old != null && !Arrays.equals(old, existing)) {
return false;
}
map.put(keyBuffer, data);
lru.remove(keyBuffer);
lru.addFirst(keyBuffer);
if (existing != null)
{
putReplaceCount++;
}
else
putAddCount++;
return true;
}
synchronized boolean remove(KeyBuffer keyBuffer)
{
byte[] old = map.remove(keyBuffer);
if (old != null)
{
boolean r = lru.remove(keyBuffer);
removeCount++;
return r;
}
return false;
}
synchronized long size()
{
return map.size();
}
static long sizeOf(KeyBuffer key, byte[] value)
{
// calculate the same value as the original impl would do
return NonMemoryPoolHashEntries.ENTRY_OFF_DATA + key.size() + value.length;
}
void resetStatistics()
{
evictedEntries = 0L;
hitCount = 0L;
missCount = 0L;
putAddCount = 0L;
putReplaceCount = 0L;
removeCount = 0L;
}
}
================================================
FILE: src/test/java/com/oath/halodb/CompactionWithErrorsTest.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import com.google.common.util.concurrent.RateLimiter;
import org.testng.Assert;
import org.testng.annotations.Test;
import sun.nio.ch.FileChannelImpl;
import java.io.IOException;
import java.nio.channels.WritableByteChannel;
import java.nio.file.Paths;
import java.util.List;
import mockit.Expectations;
import mockit.Invocation;
import mockit.Mock;
import mockit.MockUp;
import mockit.Mocked;
import mockit.VerificationsInOrder;
public class CompactionWithErrorsTest extends TestBase {
@Test
public void testCompactionWithException() throws HaloDBException, InterruptedException {
new MockUp() {
private int callCount = 0;
@Mock
public double acquire(int permits) {
if (++callCount == 3) {
// throw an exception when copying the third record.
throw new OutOfMemoryError("Throwing mock exception form compaction thread.");
}
return 10;
}
};
String directory = TestUtils.getTestDirectory("CompactionManagerTest", "testCompactionWithException");
HaloDBOptions options = new HaloDBOptions();
options.setMaxFileSize(10 * 1024);
options.setCompactionThresholdPerFile(0.5);
HaloDB db = getTestDB(directory, options);
int numberOfRecords = 30; // three files.
List records = insertAndUpdate(db, numberOfRecords);
TestUtils.waitForCompactionToComplete(db);
// An exception was thrown while copying a record in the compaction thread.
// Make sure that all records are still correct.
Assert.assertEquals(db.size(), records.size());
for (Record r : records) {
Assert.assertEquals(db.get(r.getKey()), r.getValue());
}
db.close();
db = getTestDBWithoutDeletingFiles(directory, options);
// Make sure that everything is good after
// we open the db again. Since compaction had failed
// there would be two copies of the same record in two different files.
Assert.assertEquals(db.size(), records.size());
for (Record r : records) {
Assert.assertEquals(db.get(r.getKey()), r.getValue());
}
}
@Test
public void testRestartCompactionThreadAfterCrash(@Mocked CompactionManager compactionManager) throws HaloDBException, InterruptedException, IOException {
new Expectations(CompactionManager.class) {{
// nothing mocked. call the real implementation.
// this is used only for verifications later.
}};
new MockUp() {
private int callCount = 0;
@Mock
public double acquire(int permits) {
if (++callCount == 3 || callCount == 8) {
// throw exceptions twice, each time compaction thread should crash and restart.
throw new OutOfMemoryError("Throwing mock exception from compaction thread.");
}
return 10;
}
};
String directory = TestUtils.getTestDirectory("CompactionManagerTest", "testRestartCompactionThreadAfterCrash");
HaloDBOptions options = new HaloDBOptions();
options.setMaxFileSize(10 * 1024);
options.setCompactionThresholdPerFile(0.5);
HaloDB db = getTestDB(directory, options);
int numberOfRecords = 30; // three files, 10 record in each.
List records = insertAndUpdate(db, numberOfRecords);
TestUtils.waitForCompactionToComplete(db);
// An exception was thrown while copying a record in the compaction thread.
// Make sure that all records are still correct.
Assert.assertEquals(db.size(), records.size());
for (Record r : records) {
Assert.assertEquals(db.get(r.getKey()), r.getValue());
}
db.close();
db = getTestDBWithoutDeletingFiles(directory, options);
// Make sure that everything is good after
// we open the db again. Since compaction had failed
// there would be two copies of the same record in two different files.
Assert.assertEquals(db.size(), records.size());
for (Record r : records) {
Assert.assertEquals(db.get(r.getKey()), r.getValue());
}
new VerificationsInOrder() {{
// called when db.open()
compactionManager.startCompactionThread();
// compaction thread should have crashed twice and each time it should have been restarted.
compactionManager.startCompactionThread();
compactionManager.startCompactionThread();
// called after db.close()
compactionManager.stopCompactionThread(true);
// called when db.open() the second time.
compactionManager.startCompactionThread();
}};
DBMetaData dbMetaData = new DBMetaData(dbDirectory);
dbMetaData.loadFromFileIfExists();
// Since compaction thread was restarted after it crashed IOError flag must not be set.
Assert.assertFalse(dbMetaData.isIOError());
}
@Test
public void testCompactionThreadStopWithIOException() throws HaloDBException, InterruptedException, IOException {
// Throw an IOException while stopping compaction thread.
new MockUp() {
@Mock
boolean stopCompactionThread(boolean flag) throws IOException {
throw new IOException("Throwing mock IOException while stopping compaction thread.");
}
};
String directory = TestUtils.getTestDirectory("CompactionManagerTest", "testCompactionThreadStopWithIOException");
HaloDBOptions options = new HaloDBOptions();
options.setMaxFileSize(10 * 1024);
options.setCompactionThresholdPerFile(0.5);
HaloDB db = getTestDB(directory, options);
int numberOfRecords = 20; // three files.
insertAndUpdate(db, numberOfRecords);
TestUtils.waitForCompactionToComplete(db);
db.close();
DBMetaData dbMetaData = new DBMetaData(dbDirectory);
dbMetaData.loadFromFileIfExists();
// Since there was an IOException while stopping compaction IOError flag must have been set.
Assert.assertTrue(dbMetaData.isIOError());
}
private List insertAndUpdate(HaloDB db, int numberOfRecords) throws HaloDBException {
List records = TestUtils.insertRandomRecordsOfSize(db, numberOfRecords, 1024 - Record.Header.HEADER_SIZE);
// Update first 5 records in each file.
for (int i = 0; i < 5; i++) {
byte[] value = TestUtils.generateRandomByteArray();
db.put(records.get(i).getKey(), value);
records.set(i, new Record(records.get(i).getKey(), value));
db.put(records.get(i+10).getKey(), value);
records.set(i+10, new Record(records.get(i+10).getKey(), value));
}
return records;
}
}
================================================
FILE: src/test/java/com/oath/halodb/CrossCheckTest.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import com.google.common.primitives.Longs;
import com.oath.halodb.histo.EstimatedHistogram;
import org.testng.Assert;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertTrue;
// This unit test uses the production cache implementation and an independent OHCache implementation used to
// cross-check the production implementation.
public class CrossCheckTest
{
private static final int fixedValueSize = 20;
private static final int fixedKeySize = 16;
@AfterMethod(alwaysRun = true)
public void deinit()
{
Uns.clearUnsDebugForTest();
}
static DoubleCheckOffHeapHashTableImpl cache(HashAlgorithm hashAlgorithm, boolean useMemoryPool)
{
return cache(hashAlgorithm, useMemoryPool, 256);
}
static DoubleCheckOffHeapHashTableImpl cache(HashAlgorithm hashAlgorithm, boolean useMemoryPool, long capacity)
{
return cache(hashAlgorithm, useMemoryPool, capacity, -1);
}
static DoubleCheckOffHeapHashTableImpl cache(HashAlgorithm hashAlgorithm, boolean useMemoryPool, long capacity, int hashTableSize)
{
return cache(hashAlgorithm, useMemoryPool, capacity, hashTableSize, -1, -1);
}
static DoubleCheckOffHeapHashTableImpl cache(HashAlgorithm hashAlgorithm, boolean useMemoryPool, long capacity, int hashTableSize, int segments, long maxEntrySize)
{
OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder()
.valueSerializer(HashTableTestUtils.byteArraySerializer)
.hashMode(hashAlgorithm)
.fixedValueSize(fixedValueSize);
if (useMemoryPool)
builder.useMemoryPool(true).fixedKeySize(fixedKeySize);
if (hashTableSize > 0)
builder.hashTableSize(hashTableSize);
if (segments > 0)
builder.segmentCount(segments);
else
// use 16 segments by default to prevent differing test behaviour on varying test hardware
builder.segmentCount(16);
return new DoubleCheckOffHeapHashTableImpl<>(builder);
}
@DataProvider(name = "hashAlgorithms")
public Object[][] cacheEviction()
{
return new Object[][]{
{HashAlgorithm.MURMUR3, false },
{HashAlgorithm.MURMUR3, true },
{HashAlgorithm.CRC32, false },
{HashAlgorithm.CRC32, true },
{HashAlgorithm.XX, false },
{HashAlgorithm.XX, true }
};
}
@Test(dataProvider = "hashAlgorithms")
public void testBasics(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws IOException, InterruptedException
{
try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool))
{
byte[] key = HashTableTestUtils.randomBytes(12);
byte[] value = HashTableTestUtils.randomBytes(fixedValueSize);
cache.put(key, value);
byte[] actual = cache.get(key);
Assert.assertEquals(actual, value);
cache.remove(key);
Map keyValues = new HashMap<>();
for (int i = 0; i < 100; i++) {
byte[] k = HashTableTestUtils.randomBytes(8);
byte[] v = HashTableTestUtils.randomBytes(fixedValueSize);
keyValues.put(k, v);
cache.put(k, v);
}
keyValues.forEach((k, v) -> {
Assert.assertEquals(cache.get(k), v);
});
// implicitly compares stats
cache.stats();
}
}
@Test(dataProvider = "hashAlgorithms", dependsOnMethods = "testBasics")
public void testManyValues(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws IOException, InterruptedException
{
try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool, 64, -1))
{
List entries = HashTableTestUtils.fillMany(cache, fixedValueSize);
OffHeapHashTableStats stats = cache.stats();
Assert.assertEquals(stats.getPutAddCount(), HashTableTestUtils.manyCount);
Assert.assertEquals(stats.getSize(), HashTableTestUtils.manyCount);
entries.forEach(kv -> Assert.assertEquals(cache.get(kv.key), kv.value));
stats = cache.stats();
Assert.assertEquals(stats.getHitCount(), HashTableTestUtils.manyCount);
Assert.assertEquals(stats.getSize(), HashTableTestUtils.manyCount);
for (int i = 0; i < HashTableTestUtils.manyCount; i++)
{
HashTableTestUtils.KeyValuePair kv = entries.get(i);
Assert.assertEquals(cache.get(kv.key), kv.value, "for i="+i);
assertTrue(cache.containsKey(kv.key), "for i="+i);
byte[] updated = HashTableTestUtils.randomBytes(fixedValueSize);
cache.put(kv.key, updated);
entries.set(i, new HashTableTestUtils.KeyValuePair(kv.key, updated));
Assert.assertEquals(cache.get(kv.key), updated, "for i="+i);
Assert.assertEquals(cache.size(), HashTableTestUtils.manyCount, "for i=" + i);
assertTrue(cache.containsKey(kv.key), "for i="+i);
}
stats = cache.stats();
Assert.assertEquals(stats.getPutReplaceCount(), HashTableTestUtils.manyCount);
Assert.assertEquals(stats.getSize(), HashTableTestUtils.manyCount);
entries.forEach(kv -> Assert.assertEquals(cache.get(kv.key), kv.value));
stats = cache.stats();
Assert.assertEquals(stats.getHitCount(), HashTableTestUtils.manyCount * 6);
Assert.assertEquals(stats.getSize(), HashTableTestUtils.manyCount);
for (int i = 0; i < HashTableTestUtils.manyCount; i++)
{
HashTableTestUtils.KeyValuePair kv = entries.get(i);
Assert.assertEquals(cache.get(kv.key), kv.value, "for i=" + i);
assertTrue(cache.containsKey(kv.key), "for i=" + i);
cache.remove(kv.key);
Assert.assertNull(cache.get(kv.key), "for i=" + i);
Assert.assertFalse(cache.containsKey(kv.key), "for i=" + i);
Assert.assertEquals(cache.stats().getRemoveCount(), i + 1);
Assert.assertEquals(cache.size(), HashTableTestUtils.manyCount - i - 1, "for i=" + i);
}
stats = cache.stats();
Assert.assertEquals(stats.getRemoveCount(), HashTableTestUtils.manyCount);
Assert.assertEquals(stats.getSize(), 0);
}
}
@Test(dataProvider = "hashAlgorithms", dependsOnMethods = "testBasics")
public void testRehash(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws IOException, InterruptedException
{
int count = 10_000;
OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder()
.valueSerializer(HashTableTestUtils.byteArraySerializer)
.hashMode(hashAlgorithm)
.fixedValueSize(fixedValueSize)
.hashTableSize(count/4)
.segmentCount(1)
.loadFactor(1);
if (useMemoryPool)
builder.useMemoryPool(true).fixedKeySize(fixedKeySize);
try (OffHeapHashTable cache = new DoubleCheckOffHeapHashTableImpl<>(builder))
{
List entries = HashTableTestUtils.fill(cache, fixedValueSize, count);
OffHeapHashTableStats stats = cache.stats();
Assert.assertEquals(stats.getPutAddCount(), count);
Assert.assertEquals(stats.getSize(), count);
Assert.assertEquals(stats.getRehashCount(), 2); // default load factor of 0.75, therefore 3 rehashes.
entries.forEach(kv -> Assert.assertEquals(cache.get(kv.key), kv.value));
stats = cache.stats();
Assert.assertEquals(stats.getHitCount(), count);
Assert.assertEquals(stats.getSize(), count);
for (int i = 0; i < count; i++)
{
HashTableTestUtils.KeyValuePair kv = entries.get(i);
Assert.assertEquals(cache.get(kv.key), kv.value, "for i="+i);
assertTrue(cache.containsKey(kv.key), "for i="+i);
byte[] updated = HashTableTestUtils.randomBytes(fixedValueSize);
cache.put(kv.key, updated);
entries.set(i, new HashTableTestUtils.KeyValuePair(kv.key, updated));
Assert.assertEquals(cache.get(kv.key), updated, "for i="+i);
Assert.assertEquals(cache.size(), count, "for i=" + i);
assertTrue(cache.containsKey(kv.key), "for i="+i);
}
stats = cache.stats();
Assert.assertEquals(stats.getPutReplaceCount(), count);
Assert.assertEquals(stats.getSize(), count);
Assert.assertEquals(stats.getRehashCount(), 2);
entries.forEach(kv -> Assert.assertEquals(cache.get(kv.key), kv.value));
stats = cache.stats();
Assert.assertEquals(stats.getHitCount(), count * 6);
Assert.assertEquals(stats.getSize(), count);
for (int i = 0; i < count; i++)
{
HashTableTestUtils.KeyValuePair kv = entries.get(i);
Assert.assertEquals(cache.get(kv.key), kv.value, "for i=" + i);
assertTrue(cache.containsKey(kv.key), "for i=" + i);
cache.remove(kv.key);
Assert.assertNull(cache.get(kv.key), "for i=" + i);
Assert.assertFalse(cache.containsKey(kv.key), "for i=" + i);
Assert.assertEquals(cache.stats().getRemoveCount(), i + 1);
Assert.assertEquals(cache.size(), count - i - 1, "for i=" + i);
}
stats = cache.stats();
Assert.assertEquals(stats.getRemoveCount(), count);
Assert.assertEquals(stats.getSize(), 0);
Assert.assertEquals(stats.getRehashCount(), 2);
}
}
//
// private String longString()
// {
// char[] chars = new char[900];
// for (int i = 0; i < chars.length; i++)
// chars[i] = (char) ('A' + i % 26);
// return new String(chars);
// }
//
//
@Test(dataProvider = "hashAlgorithms", dependsOnMethods = "testBasics",
expectedExceptions = IllegalArgumentException.class,
expectedExceptionsMessageRegExp = ".*greater than fixed value size.*")
public void testPutTooLargeValue(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws IOException, InterruptedException {
byte[] key = HashTableTestUtils.randomBytes(8);
byte[] largeValue = HashTableTestUtils.randomBytes(fixedValueSize + 1);
try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool, 1, -1)) {
cache.put(key, largeValue);
}
}
@Test(dataProvider = "hashAlgorithms", dependsOnMethods = "testBasics",
expectedExceptions = IllegalArgumentException.class,
expectedExceptionsMessageRegExp = ".*exceeds max permitted size of 127")
public void testPutTooLargeKey(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws IOException, InterruptedException {
byte[] key = HashTableTestUtils.randomBytes(1024);
byte[] largeValue = HashTableTestUtils.randomBytes(fixedValueSize);
try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool, 1, -1)) {
cache.put(key, largeValue);
}
}
// per-method tests
@Test(dataProvider = "hashAlgorithms", dependsOnMethods = "testBasics")
public void testAddOrReplace(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws Exception
{
try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool))
{
byte[] oldValue = null;
for (int i = 0; i < HashTableTestUtils.manyCount; i++)
assertTrue(cache.addOrReplace(Longs.toByteArray(i), oldValue, HashTableTestUtils
.randomBytes(fixedValueSize)));
byte[] key = Longs.toByteArray(42);
byte[] value = cache.get(key);
byte[] update1 = HashTableTestUtils.randomBytes(fixedValueSize);
assertTrue(cache.addOrReplace(key, value, update1));
Assert.assertEquals(cache.get(key), update1);
byte[] update2 = HashTableTestUtils.randomBytes(fixedValueSize);
assertTrue(cache.addOrReplace(key, update1, update2));
Assert.assertEquals(cache.get(key), update2);
Assert.assertFalse(cache.addOrReplace(key, update1, update2));
Assert.assertEquals(cache.get(key), update2);
cache.remove(key);
Assert.assertNull(cache.get(key));
byte[] update3 = HashTableTestUtils.randomBytes(fixedValueSize);
// update will fail since the key was removed but old value is non-null.
Assert.assertFalse(cache.addOrReplace(key, update2, update3));
Assert.assertNull(cache.get(key));
}
}
@Test(dataProvider = "hashAlgorithms")
public void testPutIfAbsent(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws Exception
{
try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool))
{
for (int i = 0; i < HashTableTestUtils.manyCount; i++)
assertTrue(cache.putIfAbsent(Longs.toByteArray(i), HashTableTestUtils.randomBytes(fixedValueSize)));
byte[] key = Longs.toByteArray(HashTableTestUtils.manyCount + 100);
byte[] value = HashTableTestUtils.randomBytes(fixedValueSize);
assertTrue(cache.putIfAbsent(key, value));
Assert.assertEquals(cache.get(key), value);
Assert.assertFalse(cache.putIfAbsent(key, value));
}
}
@Test(dataProvider = "hashAlgorithms")
public void testRemove(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws Exception
{
try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool))
{
HashTableTestUtils.fillMany(cache, fixedValueSize);
byte[] key = Longs.toByteArray(HashTableTestUtils.manyCount + 100);
byte[] value = HashTableTestUtils.randomBytes(fixedValueSize);
cache.put(key, value);
Assert.assertEquals(cache.get(key), value);
cache.remove(key);
Assert.assertNull(cache.get(key));
Assert.assertFalse(cache.remove(key));
Random r = new Random();
for (int i = 0; i < HashTableTestUtils.manyCount; i++)
cache.remove(Longs.toByteArray(r.nextInt(HashTableTestUtils.manyCount)));
}
}
@Test(dataProvider = "hashAlgorithms")
public void testClear(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws Exception
{
try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool))
{
List data = new ArrayList<>();
for (int i = 0; i < 100; i++) {
data.add(new HashTableTestUtils.KeyValuePair(Longs.toByteArray(i), HashTableTestUtils
.randomBytes(fixedValueSize)));
}
data.forEach(kv -> cache.put(kv.key, kv.value));
data.forEach(kv -> Assert.assertEquals(cache.get(kv.key), kv.value));
assertEquals(cache.size(), 100);
cache.clear();
assertEquals(cache.size(), 0);
}
}
@Test(dataProvider = "hashAlgorithms")
public void testGet_Put(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws Exception
{
try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool))
{
byte[] key = Longs.toByteArray(42);
byte[] value = HashTableTestUtils.randomBytes(fixedValueSize);
cache.put(key, value);
assertEquals(cache.get(key), value);
Assert.assertNull(cache.get(Longs.toByteArray(5)));
byte[] key11 = Longs.toByteArray(11);
byte[] value11 = HashTableTestUtils.randomBytes(fixedValueSize);
cache.put(key11, value11);
Assert.assertEquals(cache.get(key), value);
Assert.assertEquals(cache.get(key11), value11);
value11 = HashTableTestUtils.randomBytes(fixedValueSize);
cache.put(key11, value11);
Assert.assertEquals(cache.get(key), value);
Assert.assertEquals(cache.get(key11), value11);
}
}
@Test(dataProvider = "hashAlgorithms")
public void testContainsKey(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws Exception
{
try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool))
{
byte[] key = Longs.toByteArray(42);
byte[] value = HashTableTestUtils.randomBytes(fixedValueSize);
cache.put(key, value);
assertTrue(cache.containsKey(key));
Assert.assertFalse(cache.containsKey(Longs.toByteArray(11)));
}
}
@Test(dataProvider = "hashAlgorithms")
public void testGetBucketHistogram(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws Exception
{
try (DoubleCheckOffHeapHashTableImpl cache = cache(hashAlgorithm, useMemoryPool))
{
List data = new ArrayList<>();
for (int i = 0; i < 100; i++) {
data.add(new HashTableTestUtils.KeyValuePair(Longs.toByteArray(i), HashTableTestUtils
.randomBytes(fixedValueSize)));
}
data.forEach(kv -> cache.put(kv.key, kv.value));
Assert.assertEquals(cache.stats().getSize(), 100);
EstimatedHistogram hProd = cache.prod.getBucketHistogram();
Assert.assertEquals(hProd.count(), sum(cache.prod.hashTableSizes()));
long[] offsets = hProd.getBucketOffsets();
Assert.assertEquals(offsets.length, 3);
Assert.assertEquals(offsets[0], -1);
Assert.assertEquals(offsets[1], 0);
Assert.assertEquals(offsets[2], 1);
// hProd.log(LoggerFactory.getLogger(CrossCheckTest.class));
// System.out.println(Arrays.toString(offsets));
Assert.assertEquals(hProd.min(), 0);
Assert.assertEquals(hProd.max(), 1);
}
}
private static int sum(int[] ints)
{
int r = 0;
for (int i : ints)
r += i;
return r;
}
@Test(dataProvider = "hashAlgorithms")
public void testResetStatistics(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws IOException
{
try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool))
{
for (int i = 0; i < 100; i++)
cache.put(Longs.toByteArray(i), HashTableTestUtils.randomBytes(fixedValueSize));
for (int i = 0; i < 30; i++)
cache.put(Longs.toByteArray(i), HashTableTestUtils.randomBytes(fixedValueSize));
for (int i = 0; i < 50; i++)
cache.get(Longs.toByteArray(i));
for (int i = 100; i < 120; i++)
cache.get(Longs.toByteArray(i));
for (int i = 0; i < 25; i++)
cache.remove(Longs.toByteArray(i));
OffHeapHashTableStats stats = cache.stats();
Assert.assertEquals(stats.getPutAddCount(), 100);
Assert.assertEquals(stats.getPutReplaceCount(), 30);
Assert.assertEquals(stats.getHitCount(), 50);
Assert.assertEquals(stats.getMissCount(), 20);
Assert.assertEquals(stats.getRemoveCount(), 25);
cache.resetStatistics();
stats = cache.stats();
Assert.assertEquals(stats.getPutAddCount(), 0);
Assert.assertEquals(stats.getPutReplaceCount(), 0);
Assert.assertEquals(stats.getHitCount(), 0);
Assert.assertEquals(stats.getMissCount(), 0);
Assert.assertEquals(stats.getRemoveCount(), 0);
}
}
}
================================================
FILE: src/test/java/com/oath/halodb/DBDirectoryTest.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import org.hamcrest.MatcherAssert;
import org.hamcrest.Matchers;
import org.testng.Assert;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;
import java.io.File;
import java.io.FileWriter;
import java.io.IOException;
import java.io.PrintWriter;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
public class DBDirectoryTest {
private static final File directory = Paths.get("tmp", "DBDirectoryTest").toFile();
private DBDirectory dbDirectory;
private static Integer[] dataFileIds = {7, 12, 1, 8, 10};
private static Integer[] tombstoneFileIds = {21, 13, 12};
@Test
public void testListIndexFiles() {
List actual = dbDirectory.listIndexFiles();
List expected = Stream.of(dataFileIds).sorted().collect(Collectors.toList());
Assert.assertEquals(actual, expected);
Assert.assertEquals(actual.size(), dataFileIds.length);
}
@Test
public void testListDataFiles() {
File[] files = dbDirectory.listDataFiles();
List actual = Stream.of(files).map(File::getName).collect(Collectors.toList());
List expected = Stream.of(dataFileIds).map(i -> i + HaloDBFile.DATA_FILE_NAME).collect(Collectors.toList());
MatcherAssert.assertThat(actual, Matchers.containsInAnyOrder(expected.toArray()));
Assert.assertEquals(actual.size(), dataFileIds.length);
}
@Test
public void testListTombstoneFiles() {
File[] files = dbDirectory.listTombstoneFiles();
List actual = Stream.of(files).map(File::getName).collect(Collectors.toList());
List expected = Stream.of(tombstoneFileIds).sorted().map(i -> i + TombstoneFile.TOMBSTONE_FILE_NAME).collect(Collectors.toList());
Assert.assertEquals(actual, expected);
Assert.assertEquals(actual.size(), tombstoneFileIds.length);
}
@Test
public void testSyncMetaDataNoError() throws IOException {
dbDirectory.syncMetaData();
}
@BeforeMethod
public void createDirectory() throws IOException {
dbDirectory = DBDirectory.open(directory);
Path directoryPath = dbDirectory.getPath();
for (int i : dataFileIds) {
try(PrintWriter writer = new PrintWriter(new FileWriter(
directoryPath.resolve(i + IndexFile.INDEX_FILE_NAME).toString()))) {
writer.append("test");
}
try(PrintWriter writer = new PrintWriter(new FileWriter(
directoryPath.resolve(i + HaloDBFile.DATA_FILE_NAME).toString()))) {
writer.append("test");
}
}
// repair file, should be skipped.
try(PrintWriter writer = new PrintWriter(new FileWriter(
directoryPath.resolve(10000 + HaloDBFile.DATA_FILE_NAME + ".repair").toString()))) {
writer.append("test");
}
for (int i : tombstoneFileIds) {
try(PrintWriter writer = new PrintWriter(new FileWriter(
directoryPath.resolve(i + TombstoneFile.TOMBSTONE_FILE_NAME).toString()))) {
writer.append("test");
}
}
// repair file, should be skipped.
try(PrintWriter writer = new PrintWriter(new FileWriter(
directoryPath.resolve(20000 + TombstoneFile.TOMBSTONE_FILE_NAME + ".repair").toString()))) {
writer.append("test");
}
}
@AfterMethod
public void deleteDirectory() throws IOException {
dbDirectory.close();
TestUtils.deleteDirectory(directory);
}
}
================================================
FILE: src/test/java/com/oath/halodb/DBMetaDataTest.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import org.testng.Assert;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;
import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
public class DBMetaDataTest {
private static final File directory = Paths.get("tmp", "DBMetaDataTest", "testDBMetaData").toFile();
private DBDirectory dbDirectory;
@Test
public void testDBMetaData() throws IOException {
Path metaDataFile = dbDirectory.getPath().resolve(DBMetaData.METADATA_FILE_NAME);
// confirm that the file doesn't exist.
Assert.assertFalse(Files.exists(metaDataFile));
DBMetaData metaData = new DBMetaData(dbDirectory);
metaData.loadFromFileIfExists();
// file has not yet been created, return default values.
Assert.assertEquals(metaData.getVersion(), 0);
Assert.assertEquals(metaData.getMaxFileSize(), 0);
Assert.assertFalse(metaData.isOpen());
Assert.assertEquals(metaData.getSequenceNumber(), 0);
Assert.assertFalse(metaData.isIOError());
metaData.setVersion(Versions.CURRENT_META_FILE_VERSION);
metaData.setOpen(true);
metaData.setSequenceNumber(100);
metaData.setIOError(false);
metaData.setMaxFileSize(100);
metaData.storeToFile();
// confirm that the file has been created.
Assert.assertTrue(Files.exists(metaDataFile));
// load again to read stored values.
metaData = new DBMetaData(dbDirectory);
metaData.loadFromFileIfExists();
Assert.assertEquals(metaData.getVersion(), Versions.CURRENT_META_FILE_VERSION);
Assert.assertTrue(metaData.isOpen());
Assert.assertEquals(metaData.getSequenceNumber(), 100);
Assert.assertFalse(metaData.isIOError());
Assert.assertEquals(metaData.getMaxFileSize(), 100);
metaData.setVersion(Versions.CURRENT_META_FILE_VERSION + 10);
metaData.setOpen(false);
metaData.setSequenceNumber(Long.MAX_VALUE);
metaData.setIOError(true);
metaData.setMaxFileSize(1024);
metaData.storeToFile();
// load again to read stored values.
metaData = new DBMetaData(dbDirectory);
metaData.loadFromFileIfExists();
Assert.assertEquals(metaData.getVersion(), Versions.CURRENT_META_FILE_VERSION + 10);
Assert.assertFalse(metaData.isOpen());
Assert.assertEquals(metaData.getSequenceNumber(), Long.MAX_VALUE);
Assert.assertTrue(metaData.isIOError());
Assert.assertEquals(metaData.getMaxFileSize(), 1024);
}
@Test
public void testCheckSum() throws IOException {
DBMetaData metaData = new DBMetaData(dbDirectory);
metaData.loadFromFileIfExists();
metaData.setVersion(Versions.CURRENT_META_FILE_VERSION);
metaData.setOpen(true);
metaData.setSequenceNumber(100);
metaData.setIOError(false);
metaData.setMaxFileSize(100);
metaData.storeToFile();
metaData = new DBMetaData(dbDirectory);
metaData.loadFromFileIfExists();
Assert.assertTrue(metaData.isValid());
}
@BeforeMethod
public void createDirectory() throws IOException {
dbDirectory = DBDirectory.open(directory);
}
@AfterMethod
public void deleteDirectory() throws IOException {
dbDirectory.close();
TestUtils.deleteDirectory(directory);
}
}
================================================
FILE: src/test/java/com/oath/halodb/DBRepairTest.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import org.testng.Assert;
import org.testng.annotations.Test;
import java.io.File;
import java.io.IOException;
import java.nio.file.attribute.FileTime;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
public class DBRepairTest extends TestBase {
@Test(dataProvider = "Options")
public void testRepairDB(HaloDBOptions options) throws HaloDBException, IOException {
String directory = TestUtils.getTestDirectory("DBRepairTest", "testRepairDB");
options.setMaxFileSize(1024 * 1024);
options.setCompactionDisabled(true);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 5 * 1024 + 512; // 5 files with 1024 records and 1 with 512 records.
List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-Record.Header.HEADER_SIZE);
// delete half the records.
for (int i = 0; i < noOfRecords; i++) {
if (i % 2 == 0) {
db.delete(records.get(i).getKey());
}
}
FileTime latestDataFileCreatedTime =
TestUtils.getFileCreationTime(TestUtils.getLatestDataFile(directory).get());
FileTime latestTombstoneFileCreationTime =
TestUtils.getFileCreationTime(FileUtils.listTombstoneFiles(new File(directory))[0]);
db.close();
// trick the db to think that there was an unclean shutdown.
DBMetaData dbMetaData = new DBMetaData(dbDirectory);
dbMetaData.setOpen(true);
dbMetaData.storeToFile();
// wait for a second so that the new file will have a different create at time.
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
}
db = getTestDBWithoutDeletingFiles(directory, options);
// latest file should have been repaired and replaced.
Assert.assertNotEquals(
TestUtils.getFileCreationTime(TestUtils.getLatestDataFile(directory).get()),
latestDataFileCreatedTime
);
Assert.assertNotEquals(
TestUtils.getFileCreationTime(FileUtils.listTombstoneFiles(new File(directory))[0]),
latestTombstoneFileCreationTime
);
Assert.assertEquals(db.size(), noOfRecords/2);
for (int i = 0; i < noOfRecords; i++) {
if (i % 2 == 0) {
Assert.assertNull(db.get(records.get(i).getKey()));
}
else {
Record r = records.get(i);
Assert.assertEquals(db.get(r.getKey()), r.getValue());
}
}
}
@Test(dataProvider = "Options")
public void testRepairDBWithCompaction(HaloDBOptions options) throws HaloDBException, InterruptedException, IOException {
String directory = TestUtils.getTestDirectory("DBRepairTest", "testRepairDBWithCompaction");
options.setMaxFileSize(1024 * 1024);
options.setCompactionThresholdPerFile(0.5);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10 * 1024 + 512;
List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-Record.Header.HEADER_SIZE);
List toUpdate = IntStream.range(0, noOfRecords).filter(i -> i%2==0).mapToObj(i -> records.get(i)).collect(Collectors.toList());
List updatedRecords = TestUtils.updateRecords(db, toUpdate);
for (int i = 0; i < updatedRecords.size(); i++) {
records.set(i * 2, updatedRecords.get(i));
}
TestUtils.waitForCompactionToComplete(db);
FileTime latestDataFileCreatedTime =
TestUtils.getFileCreationTime(TestUtils.getLatestDataFile(directory).get());
db.close();
// trick the db to think that there was an unclean shutdown.
DBMetaData dbMetaData = new DBMetaData(dbDirectory);
dbMetaData.setOpen(true);
dbMetaData.storeToFile();
// wait for a second so that the new file will have a different created at time.
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
}
db = getTestDBWithoutDeletingFiles(directory, options);
// latest file should have been repaired and replaced.
Assert.assertNotEquals(
TestUtils.getFileCreationTime(TestUtils.getLatestDataFile(directory).get()),
latestDataFileCreatedTime
);
Assert.assertEquals(db.size(), noOfRecords);
for (Record r : records) {
Assert.assertEquals(db.get(r.getKey()), r.getValue());
}
}
@Test
public void testRepairWithMultipleTombstoneFiles() throws HaloDBException, IOException {
String directory = TestUtils.getTestDirectory("DBRepairTest", "testRepairWithMultipleTombstoneFiles");
HaloDBOptions options = new HaloDBOptions();
options.setCompactionDisabled(true);
options.setMaxFileSize(320);
HaloDB db = getTestDB(directory, options);
int noOfTombstonesPerFile = 10;
int noOfFiles = 3;
int noOfRecords = noOfTombstonesPerFile * noOfFiles;
// Since keyLength was 19 tombstone entry is 32 bytes.
int keyLength = 19;
int valueLength = 24;
List records = new ArrayList<>();
for (int i = 0; i < noOfRecords; i++) {
Record r = new Record(TestUtils.generateRandomByteArray(keyLength), TestUtils.generateRandomByteArray(valueLength));
records.add(r);
db.put(r.getKey(), r.getValue());
}
for (Record r : records) {
db.delete(r.getKey());
}
File[] tombstoneFiles = FileUtils.listTombstoneFiles(new File(directory));
FileTime latestDataFileCreatedTime =
TestUtils.getFileCreationTime(TestUtils.getLatestDataFile(directory).get());
FileTime latestTombstoneFileCreationTime =
TestUtils.getFileCreationTime(tombstoneFiles[tombstoneFiles.length-1]);
db.close();
// trick the db to think that there was an unclean shutdown.
DBMetaData dbMetaData = new DBMetaData(dbDirectory);
dbMetaData.setOpen(true);
dbMetaData.storeToFile();
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
}
db = getTestDBWithoutDeletingFiles(directory, options);
// latest file should have been repaired and replaced.
Assert.assertNotEquals(
TestUtils.getFileCreationTime(TestUtils.getLatestDataFile(directory).get()),
latestDataFileCreatedTime
);
Assert.assertNotEquals(
TestUtils.getFileCreationTime(tombstoneFiles[tombstoneFiles.length-1]),
latestTombstoneFileCreationTime
);
// other two tombstone files should still be there
Assert.assertTrue(tombstoneFiles[0].exists());
Assert.assertTrue(tombstoneFiles[1].exists());
Assert.assertEquals(db.size(), 0);
for (int i = 0; i < noOfRecords; i++) {
Assert.assertNull(db.get(records.get(i).getKey()));
}
}
}
================================================
FILE: src/test/java/com/oath/halodb/DataConsistencyDB.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.locks.ReentrantReadWriteLock;
/**
* Holds an instance of HaloDB and Java's ConcurrentHashMap.
* Tests will use this to insert data into both and ensure that
* the data in HaloDB is correct.
*/
class DataConsistencyDB {
private static final Logger logger = LoggerFactory.getLogger(DataConsistencyDB.class);
//TODO: allocate this off-heap.
private final Map javaMap = new ConcurrentHashMap<>();
private final HaloDB haloDB;
private int numberOfLocks = 100;
private final ReentrantReadWriteLock[] locks;
DataConsistencyDB(HaloDB haloDB, int noOfRecords) {
this.haloDB = haloDB;
locks = new ReentrantReadWriteLock[numberOfLocks];
for (int i = 0; i < numberOfLocks; i++) {
locks[i] = new ReentrantReadWriteLock();
}
}
void put(int keyIndex, ByteBuffer keyBuf, byte[] value) throws HaloDBException {
ReentrantReadWriteLock lock = locks[keyIndex%numberOfLocks];
try {
lock.writeLock().lock();
javaMap.put(keyBuf, value);
haloDB.put(keyBuf.array(), value);
}
finally {
lock.writeLock().unlock();
}
}
// return -1 if values don't match.
int compareValues(int keyIndex, ByteBuffer keyBuf) throws HaloDBException {
ReentrantReadWriteLock lock = locks[keyIndex%numberOfLocks];
try {
lock.readLock().lock();
return checkValues(keyIndex, keyBuf, haloDB);
}
finally {
lock.readLock().unlock();
}
}
boolean checkSize() {
return haloDB.size() == javaMap.size();
}
void delete(int keyIndex, ByteBuffer keyBuf) throws HaloDBException {
ReentrantReadWriteLock lock = locks[keyIndex%numberOfLocks];
try {
lock.writeLock().lock();
javaMap.remove(keyBuf);
haloDB.delete(keyBuf.array());
}
finally {
lock.writeLock().unlock();
}
}
boolean iterateAndCheck(HaloDB db) {
if (db.size() != javaMap.size()) {
logger.error("Size don't match {} != {}", db.size(), javaMap.size());
return false;
}
for (Map.Entry entry : javaMap.entrySet()) {
try {
if (!Arrays.equals(entry.getValue(), db.get(entry.getKey().array()))) {
return false;
}
} catch (HaloDBException e) {
logger.error("Error while iterating", e);
return false;
}
}
return true;
}
// return -1 if values don't match.
private int checkValues(long key, ByteBuffer keyBuf, HaloDB haloDB) throws HaloDBException {
byte[] mapValue = javaMap.get(keyBuf);
byte[] dbValue = haloDB.get(keyBuf.array());
if (Arrays.equals(mapValue, dbValue))
return dbValue == null ? 0 : dbValue.length;
if (mapValue == null) {
logger.error("Map value is null for key {} of length {} but HaloDB value has version {}",
key, keyBuf.remaining(), DataConsistencyTest.getVersionFromValue(dbValue));
}
else if (dbValue == null) {
logger.error("HaloDB value is null for key {} of length {} but Map value has version {}",
key, keyBuf.remaining(), DataConsistencyTest.getVersionFromValue(mapValue));
}
else {
logger.error("HaloDB value for key {} has version {} of length {} but map value version is {}",
key, keyBuf.remaining(), DataConsistencyTest.getVersionFromValue(dbValue), DataConsistencyTest
.getVersionFromValue(mapValue));
}
return -1;
}
boolean containsKey(byte[] key) throws HaloDBException {
return haloDB.get(key) != null;
}
}
================================================
FILE: src/test/java/com/oath/halodb/DataConsistencyTest.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import com.google.common.primitives.Longs;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.Assert;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;
import java.nio.ByteBuffer;
import java.util.HashSet;
import java.util.Random;
import java.util.Set;
public class DataConsistencyTest extends TestBase {
private static final Logger logger = LoggerFactory.getLogger(DataConsistencyTest.class);
private final Object lock = new Object();
private volatile boolean insertionComplete;
private volatile boolean updatesComplete;
private volatile boolean foundNonMatchingValue;
private static final int fixedKeySize = 16;
private static final int maxValueSize = 100;
private static final int noOfRecords = 100_000;
private static final int noOfTransactions = 1_000_000;
private ByteBuffer[] keys;
private RandomDataGenerator randDataGenerator;
private Random random = new Random();
private HaloDB haloDB;
@BeforeMethod
public void init() {
insertionComplete = false;
updatesComplete = false;
foundNonMatchingValue = false;
keys = new ByteBuffer[noOfRecords];
randDataGenerator = new RandomDataGenerator();
}
@Test(dataProvider = "Options")
public void testConcurrentReadAndUpdates(HaloDBOptions options) throws HaloDBException, InterruptedException {
String directory = TestUtils.getTestDirectory("DataConsistencyCheck", "testConcurrentReadAndUpdates");
options.setMaxFileSize(10 * 1024);
options.setCompactionThresholdPerFile(0.1);
options.setFixedKeySize(fixedKeySize);
options.setNumberOfRecords(2 * noOfRecords);
haloDB = getTestDB(directory, options);
DataConsistencyDB db = new DataConsistencyDB(haloDB, noOfRecords);
Writer writer = new Writer(db);
writer.start();
synchronized (lock) {
while (!insertionComplete) {
lock.wait();
}
}
long start = System.currentTimeMillis();
Reader[] readers = new Reader[10];
for (int i = 0; i < readers.length; i++) {
readers[i] = new Reader(db);
readers[i].start();
}
writer.join();
long totalReads = 0, totalReadSize = 0;
for (Reader reader : readers) {
reader.join();
totalReads += reader.readCount;
totalReadSize += reader.readSize;
}
long time = (System.currentTimeMillis() - start)/1000;
Assert.assertFalse(foundNonMatchingValue);
Assert.assertTrue(db.checkSize());
haloDB.close();
logger.info("Iterating and checking ...");
HaloDB openAgainDB = getTestDBWithoutDeletingFiles(directory, options);
TestUtils.waitForCompactionToComplete(openAgainDB);
Assert.assertTrue(db.iterateAndCheck(openAgainDB));
logger.info("Completed {} updates", writer.updateCount);
logger.info("Completed {} deletes", writer.deleteCount);
logger.info("Completed {} reads", totalReads);
logger.info("Reads per second {}. {} MB/second", totalReads/time, totalReadSize/1024/1024/time);
logger.info("Writes per second {}. {} KB/second", noOfTransactions/time, writer.totalWriteSize/1024/time);
logger.info("Compaction rate {} KB/second", haloDB.stats().getCompactionRateSinceBeginning()/1024);
}
class Writer extends Thread {
DataConsistencyDB db;
long updateCount = 0;
long deleteCount = 0;
volatile long totalWriteSize = 0;
Set deletedKeys = new HashSet<>(50_000);
Writer(DataConsistencyDB db) {
this.db = db;
setPriority(MAX_PRIORITY);
}
@Override
public void run() {
Random random = new Random();
try {
for (int i = 0; i < noOfRecords; i++) {
try {
byte[] key = randDataGenerator.getData(getRandomKeyLength());
while (db.containsKey(key)) {
key = randDataGenerator.getData(getRandomKeyLength());
}
keys[i] = ByteBuffer.wrap(key);
// we need at least 8 bytes for the version.
int size = random.nextInt(maxValueSize) + 9;
db.put(i, keys[i], generateRandomValueWithVersion(updateCount, size));
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
}
} finally {
synchronized (lock) {
insertionComplete = true;
lock.notify();
}
}
try {
while (!foundNonMatchingValue && updateCount < noOfTransactions) {
int k = random.nextInt(noOfRecords);
int size = random.nextInt(maxValueSize) + 9;
updateCount++;
try {
if (updateCount % 2 == 0) {
db.delete(k, keys[k]);
deleteCount++;
deletedKeys.add(k);
if (deletedKeys.size() == 50_000) {
int keyToAdd = deletedKeys.iterator().next();
db.put(keyToAdd, keys[keyToAdd], generateRandomValueWithVersion(updateCount, size));
totalWriteSize += size;
deletedKeys.remove(keyToAdd);
}
}
else {
db.put(k, keys[k], generateRandomValueWithVersion(updateCount, size));
totalWriteSize += size;
}
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
if (updateCount > 0 && updateCount % 500_000 == 0) {
logger.info("Completed {} updates", updateCount);
}
}
} finally {
updatesComplete = true;
}
}
}
class Reader extends Thread {
DataConsistencyDB db;
volatile long readCount = 0;
volatile long readSize = 0;
Reader(DataConsistencyDB db) {
this.db = db;
setPriority(MIN_PRIORITY);
}
@Override
public void run() {
Random random = new Random();
while (!updatesComplete) {
int i = random.nextInt(noOfRecords);
try {
int valueSize = db.compareValues(i, keys[i]);
readCount++;
if (valueSize == -1) {
foundNonMatchingValue = true;
}
Assert.assertNotEquals(valueSize, -1, "Values don't match for key " + i);
readSize += valueSize;
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
}
}
}
private int getRandomKeyLength() {
return random.nextInt(fixedKeySize) + 1;
}
private byte[] generateRandomValueWithVersion(long version, int size) {
byte[] value = randDataGenerator.getData(size);
System.arraycopy(Longs.toByteArray(version), 0, value, size - 8, 8);
return value;
}
static long getVersionFromValue(byte[] value) {
byte[] v = new byte[8];
System.arraycopy(value, value.length-8, v, 0, 8);
return Longs.fromByteArray(v);
}
}
================================================
FILE: src/test/java/com/oath/halodb/DoubleCheckOffHeapHashTableImpl.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import com.google.common.primitives.Longs;
import com.oath.halodb.histo.EstimatedHistogram;
import org.testng.Assert;
import java.io.IOException;
/**
* Test code that contains an instance of the production and check {@link OffHeapHashTable}
* implementations {@link OffHeapHashTableImpl} and
* {@link CheckOffHeapHashTable}.
*/
public class DoubleCheckOffHeapHashTableImpl implements OffHeapHashTable
{
public final OffHeapHashTable prod;
public final OffHeapHashTable check;
public DoubleCheckOffHeapHashTableImpl(OffHeapHashTableBuilder builder)
{
this.prod = builder.build();
this.check = new CheckOffHeapHashTable<>(builder);
}
public boolean put(byte[] key, V value)
{
boolean rProd = prod.put(key, value);
boolean rCheck = check.put(key, value);
Assert.assertEquals(rProd, rCheck, "for key='" + key + '\'');
return rProd;
}
public boolean addOrReplace(byte[] key, V old, V value)
{
boolean rProd = prod.addOrReplace(key, old, value);
boolean rCheck = check.addOrReplace(key, old, value);
Assert.assertEquals(rProd, rCheck, "for key='" + key + '\'');
return rProd;
}
public boolean putIfAbsent(byte[] k, V v)
{
boolean rProd = prod.putIfAbsent(k, v);
boolean rCheck = check.putIfAbsent(k, v);
Assert.assertEquals(rProd, rCheck, "for key='" + k + '\'');
return rProd;
}
public boolean putIfAbsent(byte[] key, V value, long expireAt)
{
throw new UnsupportedOperationException();
}
public boolean put(byte[] key, V value, long expireAt)
{
throw new UnsupportedOperationException();
}
public boolean remove(byte[] key)
{
boolean rProd = prod.remove(key);
boolean rCheck = check.remove(key);
Assert.assertEquals(rCheck, rProd, "for key='" + key + '\'');
return rProd;
}
public void clear()
{
prod.clear();
check.clear();
}
public V get(byte[] key)
{
V rProd = prod.get(key);
V rCheck = check.get(key);
Assert.assertEquals(rProd, rCheck, "for key='" + Longs.fromByteArray(key) + '\'');
return rProd;
}
public boolean containsKey(byte[] key)
{
boolean rProd = prod.containsKey(key);
boolean rCheck = check.containsKey(key);
Assert.assertEquals(rProd, rCheck, "for key='" + key + '\'');
return rProd;
}
public void resetStatistics()
{
prod.resetStatistics();
check.resetStatistics();
}
public long size()
{
long rProd = prod.size();
long rCheck = check.size();
Assert.assertEquals(rProd, rCheck);
return rProd;
}
public int[] hashTableSizes()
{
return prod.hashTableSizes();
}
@Override
public SegmentStats[] perSegmentStats() {
SegmentStats[] rProd = prod.perSegmentStats();
SegmentStats[] rCheck = check.perSegmentStats();
Assert.assertEquals(rProd, rCheck);
return rProd;
}
public EstimatedHistogram getBucketHistogram()
{
return prod.getBucketHistogram();
}
public int segments()
{
int rProd = prod.segments();
int rCheck = check.segments();
Assert.assertEquals(rProd, rCheck);
return rProd;
}
public float loadFactor()
{
float rProd = prod.loadFactor();
float rCheck = check.loadFactor();
Assert.assertEquals(rProd, rCheck);
return rProd;
}
public OffHeapHashTableStats stats()
{
OffHeapHashTableStats rProd = prod.stats();
OffHeapHashTableStats rCheck = check.stats();
Assert.assertEquals(rProd, rCheck);
return rProd;
}
public void close() throws IOException
{
prod.close();
check.close();
}
}
================================================
FILE: src/test/java/com/oath/halodb/FileUtilsTest.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import java.nio.file.Files;
import java.nio.file.Path;
import org.hamcrest.MatcherAssert;
import org.hamcrest.Matchers;
import org.testng.Assert;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;
import java.io.File;
import java.io.FileWriter;
import java.io.IOException;
import java.io.PrintWriter;
import java.nio.file.Paths;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
public class FileUtilsTest {
private String directory = TestUtils.getTestDirectory("FileUtilsTest");
private Integer[] fileIds = {7, 12, 1, 8, 10};
private List indexFileNames =
Stream.of(fileIds)
.map(i -> Paths.get(directory).resolve(i + IndexFile.INDEX_FILE_NAME).toString())
.collect(Collectors.toList());
private List dataFileNames =
Stream.of(fileIds)
.map(i -> Paths.get(directory).resolve(i + HaloDBFile.DATA_FILE_NAME).toString())
.collect(Collectors.toList());
private List dataFileNamesRepair =
Stream.of(fileIds)
.map(i -> Paths.get(directory).resolve(i + HaloDBFile.DATA_FILE_NAME + ".repair").toString())
.collect(Collectors.toList());
private List tombstoneFileNames =
Stream.of(fileIds)
.map(i -> Paths.get(directory).resolve(i + TombstoneFile.TOMBSTONE_FILE_NAME).toString())
.collect(Collectors.toList());
@BeforeMethod
public void createDirectory() throws IOException {
TestUtils.deleteDirectory(new File(directory));
FileUtils.createDirectoryIfNotExists(new File(directory));
for (String f : indexFileNames) {
try(PrintWriter writer = new PrintWriter(new FileWriter(f))) {
writer.append("test");
}
}
for (String f : dataFileNames) {
try(PrintWriter writer = new PrintWriter(new FileWriter(f))) {
writer.append("test");
}
}
for (String f : dataFileNamesRepair) {
try(PrintWriter writer = new PrintWriter(new FileWriter(f))) {
writer.append("test");
}
}
for (String f : tombstoneFileNames) {
try(PrintWriter writer = new PrintWriter(new FileWriter(f))) {
writer.append("test");
}
}
}
@AfterMethod
public void deleteDirectory() throws IOException {
TestUtils.deleteDirectory(new File(directory));
}
@Test
public void testListIndexFiles() {
List actual = FileUtils.listIndexFiles(new File(directory));
List expected = Stream.of(fileIds).sorted().collect(Collectors.toList());
Assert.assertEquals(actual, expected);
}
@Test
public void testListDataFiles() {
File[] files = FileUtils.listDataFiles(new File(directory));
List actual = Stream.of(files).map(File::getName).collect(Collectors.toList());
List expected = Stream.of(fileIds).map(i -> i + HaloDBFile.DATA_FILE_NAME).collect(Collectors.toList());
MatcherAssert.assertThat(actual, Matchers.containsInAnyOrder(expected.toArray()));
}
@Test
public void testListTombstoneFiles() {
File[] files = FileUtils.listTombstoneFiles(new File(directory));
List actual = Stream.of(files).map(File::getName).collect(Collectors.toList());
List expected = Stream.of(fileIds).sorted().map(i -> i + TombstoneFile.TOMBSTONE_FILE_NAME).collect(Collectors.toList());
Assert.assertEquals(actual, expected);
}
@Test
public void testDirectoryCreateAndDelete() throws IOException {
Path parentDir = TestUtils.getTestDirectoryPath("FileUtilsTest", "DirectoryCreateAndDelete");
FileUtils.createDirectoryIfNotExists(parentDir.toFile());
Path subDir = TestUtils.getTestDirectoryPath("FileUtilsTest", "DirectoryCreateAndDelete", "subDir");
FileUtils.createDirectoryIfNotExists(subDir.toFile());
FileUtils.deleteDirectory(parentDir.toFile());
Assert.assertFalse(Files.exists(parentDir));
}
}
================================================
FILE: src/test/java/com/oath/halodb/HaloDBCompactionTest.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import com.google.common.primitives.Longs;
import org.testng.Assert;
import org.testng.annotations.Test;
import java.io.File;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
public class HaloDBCompactionTest extends TestBase {
private final int recordSize = 1024;
private final int numberOfFiles = 8;
private final int recordsPerFile = 10;
private final int numberOfRecords = numberOfFiles * recordsPerFile;
@Test(dataProvider = "Options")
public void testCompaction(HaloDBOptions options) throws Exception {
String directory = TestUtils.getTestDirectory("HaloDBCompactionTest", "testCompaction");
options.setMaxFileSize(recordsPerFile * recordSize);
options.setCompactionThresholdPerFile(0.5);
options.setFlushDataSizeBytes(2048);
HaloDB db = getTestDB(directory, options);
Record[] records = insertAndUpdateRecords(numberOfRecords, db);
TestUtils.waitForCompactionToComplete(db);
for (Record r : records) {
byte[] actual = db.get(r.getKey());
Assert.assertEquals(r.getValue(), actual);
}
}
@Test(dataProvider = "Options")
public void testReOpenDBAfterCompaction(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBCompactionTest", "testReOpenDBAfterCompaction");
options.setMaxFileSize(recordsPerFile * recordSize);
options.setCompactionThresholdPerFile(0.5);
HaloDB db = getTestDB(directory, options);
Record[] records = insertAndUpdateRecords(numberOfRecords, db);
TestUtils.waitForCompactionToComplete(db);
db.close();
db = getTestDBWithoutDeletingFiles(directory, options);
for (Record r : records) {
byte[] actual = db.get(r.getKey());
Assert.assertEquals(actual, r.getValue());
}
}
@Test(dataProvider = "Options")
public void testReOpenDBWithoutMerge(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBCompactionTest", "testReOpenAndUpdatesAndWithoutMerge");
options.setMaxFileSize(recordsPerFile * recordSize);
options.setCompactionDisabled(true);
HaloDB db = getTestDB(directory, options);
Record[] records = insertAndUpdateRecords(numberOfRecords, db);
db.close();
db = getTestDBWithoutDeletingFiles(directory, options);
for (Record r : records) {
byte[] actual = db.get(r.getKey());
Assert.assertEquals(actual, r.getValue());
}
}
@Test(dataProvider = "Options")
public void testSyncWrites(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBCompactionTest", "testSyncWrites");
options.enableSyncWrites(true);
HaloDB db = getTestDB(directory, options);
List records = TestUtils.insertRandomRecords(db, 10_000);
List current = new ArrayList<>();
for (int i = 0; i < records.size(); i++) {
if (i % 2 == 0) {
db.delete(records.get(i).getKey());
}
else {
current.add(records.get(i));
}
}
db.close();
db = getTestDBWithoutDeletingFiles(directory, options);
Assert.assertEquals(db.size(), current.size());
for (Record r : current) {
Assert.assertEquals(db.get(r.getKey()), r.getValue());
}
}
@Test(dataProvider = "Options")
public void testUpdatesToSameFile(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBCompactionTest", "testUpdatesToSameFile");
options.setMaxFileSize(recordsPerFile * recordSize);
options.setCompactionDisabled(true);
HaloDB db = getTestDB(directory, options);
Record[] records = insertAndUpdateRecordsToSameFile(2, db);
db.close();
db = getTestDBWithoutDeletingFiles(directory, options);
for (Record r : records) {
byte[] actual = db.get(r.getKey());
Assert.assertEquals(actual, r.getValue());
}
}
@Test(dataProvider = "Options")
public void testFilesWithStaleDataAddedToCompactionQueueDuringDBOpen(HaloDBOptions options) throws HaloDBException, InterruptedException {
String directory = TestUtils.getTestDirectory("HaloDBCompactionTest", "testFilesWithStaleDataAddedToCompactionQueueDuringDBOpen");
options.setCompactionDisabled(true);
options.setMaxFileSize(10 * 1024);
HaloDB db = getTestDB(directory, options);
// insert 50 records into 5 files.
List records = TestUtils.insertRandomRecordsOfSize(db, 50, 1024-Record.Header.HEADER_SIZE);
// Delete all records, which means that all data files would have crossed the
// stale data threshold.
for (Record r : records) {
db.delete(r.getKey());
}
db.close();
// open the db withe compaction enabled.
options.setCompactionDisabled(false);
options.setMaxFileSize(10 * 1024);
db = getTestDBWithoutDeletingFiles(directory, options);
TestUtils.waitForCompactionToComplete(db);
// Since all files have crossed stale data threshold, everything will be compacted and deleted.
Assert.assertFalse(TestUtils.getLatestDataFile(directory).isPresent());
Assert.assertFalse(TestUtils.getLatestCompactionFile(directory).isPresent());
db.close();
// open the db with compaction disabled.
options.setMaxFileSize(10 * 1024);
options.setCompactionDisabled(true);
db = getTestDBWithoutDeletingFiles(directory, options);
// insert 20 records into two files.
records = TestUtils.insertRandomRecordsOfSize(db, 20, 1024-Record.Header.HEADER_SIZE);
File[] dataFilesToDelete = FileUtils.listDataFiles(new File(directory));
// update all records; since compaction is disabled no file is deleted.
List updatedRecords = TestUtils.updateRecords(db, records);
db.close();
// Open db again with compaction enabled.
options.setCompactionDisabled(false);
options.setMaxFileSize(10 * 1024);
db = getTestDBWithoutDeletingFiles(directory, options);
TestUtils.waitForCompactionToComplete(db);
//Confirm that previous data files were compacted and deleted.
for (File f : dataFilesToDelete) {
Assert.assertFalse(f.exists());
}
for (Record r : updatedRecords) {
Assert.assertEquals(db.get(r.getKey()), r.getValue());
}
}
@Test
public void testPauseAndResumeCompaction() throws HaloDBException, InterruptedException {
String directory = TestUtils.getTestDirectory("HaloDBCompactionTest", "testPauseAndResumeCompaction");
HaloDBOptions options = new HaloDBOptions();
options.setMaxFileSize(10 * 1024);
// start compaction immediately after a record is updated.
options.setCompactionThresholdPerFile(.001);
HaloDB db = getTestDB(directory, options);
// insert 100 records of size 1kb into 100 files.
int noOfRecords = 1000;
List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024 - Record.Header.HEADER_SIZE);
List dataFiles = TestUtils.getDataFiles(directory);
db.pauseCompaction();
// update first record of each file.
List recordsToUpdate = IntStream.range(0, records.size()).filter(i -> i%10 == 0)
.mapToObj(i -> records.get(i)).collect(Collectors.toList());
TestUtils.updateRecordsWithSize(db, recordsToUpdate, 1024-Record.Header.HEADER_SIZE);
TestUtils.waitForCompactionToComplete(db);
// compaction was paused, therefore no compaction files must be present.
Assert.assertFalse(TestUtils.getLatestCompactionFile(directory).isPresent());
// resume and pause compaction a few times.
// each is also called multiple times; duplicate calls shouldn't have any effect.
db.resumeCompaction();
Assert.assertTrue(db.stats().isCompactionRunning());
Thread.sleep(5);
db.pauseCompaction();
db.pauseCompaction();
Assert.assertFalse(db.stats().isCompactionRunning());
TestUtils.waitForCompactionToComplete(db);
Thread.sleep(100);
db.resumeCompaction();
db.resumeCompaction();
Assert.assertTrue(db.stats().isCompactionRunning());
Thread.sleep(20);
db.pauseCompaction();
db.pauseCompaction();
db.pauseCompaction();
Assert.assertFalse(db.stats().isCompactionRunning());
TestUtils.waitForCompactionToComplete(db);
Thread.sleep(100);
db.resumeCompaction();
db.resumeCompaction();
Assert.assertTrue(db.stats().isCompactionRunning());
TestUtils.waitForCompactionToComplete(db);
// compaction files are present.
Assert.assertTrue(TestUtils.getLatestCompactionFile(directory).isPresent());
// all the data files created before update were deleted by compaction thread.
dataFiles.forEach(f -> Assert.assertFalse(f.exists(), "data file " + f.getName() + " still exists"));
}
private Record[] insertAndUpdateRecords(int numberOfRecords, HaloDB db) throws HaloDBException {
int valueSize = recordSize - Record.Header.HEADER_SIZE - 8; // 8 is the key size.
Record[] records = new Record[numberOfRecords];
for (int i = 0; i < numberOfRecords; i++) {
byte[] key = Longs.toByteArray(i);
byte[] value = TestUtils.generateRandomByteArray(valueSize);
records[i] = new Record(key, value);
db.put(records[i].getKey(), records[i].getValue());
}
// modify first 5 records of each file.
byte[] modifiedMark = "modified".getBytes();
for (int k = 0; k < numberOfFiles; k++) {
for (int i = 0; i < 5; i++) {
Record r = records[i + k*10];
byte[] value = r.getValue();
System.arraycopy(modifiedMark, 0, value, 0, modifiedMark.length);
Record modifiedRecord = new Record(r.getKey(), value);
records[i + k*10] = modifiedRecord;
db.put(modifiedRecord.getKey(), modifiedRecord.getValue());
}
}
return records;
}
private Record[] insertAndUpdateRecordsToSameFile(int numberOfRecords, HaloDB db) throws HaloDBException {
int valueSize = recordSize - Record.Header.HEADER_SIZE - 8; // 8 is the key size.
Record[] records = new Record[numberOfRecords];
for (int i = 0; i < numberOfRecords; i++) {
byte[] key = Longs.toByteArray(i);
byte[] value = TestUtils.generateRandomByteArray(valueSize);
byte[] updatedValue = null;
for (long j = 0; j < recordsPerFile; j++) {
updatedValue = TestUtils.concatenateArrays(value, Longs.toByteArray(i));
db.put(key, updatedValue);
}
// only store the last updated valued.
records[i] = new Record(key, updatedValue);
}
return records;
}
}
================================================
FILE: src/test/java/com/oath/halodb/HaloDBDeletionTest.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import org.testng.Assert;
import org.testng.annotations.Test;
import java.io.File;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Random;
import java.util.Set;
public class HaloDBDeletionTest extends TestBase {
@Test(dataProvider = "Options")
public void testSimpleDelete(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBDeletionTest", "testSimpleDelete");
options.setCompactionDisabled(true);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
// delete every other record.
for (int i = 0; i < records.size(); i++) {
if (i % 2 == 0) {
db.delete(records.get(i).getKey());
}
}
for (int i = 0; i < records.size(); i++) {
byte[] actual = db.get(records.get(i).getKey());
if (i % 2 == 0) {
Assert.assertNull(actual);
}
else {
Assert.assertEquals(records.get(i).getValue(), actual);
}
}
}
@Test(dataProvider = "Options")
public void testDeleteWithIterator(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBDeletionTest", "testDeleteWithIterator");
options.setCompactionDisabled(true);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
// delete every other record.
List expected = new ArrayList<>();
for (int i = 0; i < records.size(); i++) {
if (i % 2 == 0) {
db.delete(records.get(i).getKey());
}
else {
expected.add(records.get(i));
}
}
List actual = new ArrayList<>();
db.newIterator().forEachRemaining(actual::add);
Assert.assertTrue(actual.containsAll(expected) && expected.containsAll(actual));
}
@Test(dataProvider = "Options")
public void testDeleteAndInsert(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBDeletionTest", "testDeleteAndInsert");
options.setCompactionDisabled(true);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 100;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
// delete every other record.
for (int i = 0; i < records.size(); i++) {
if (i % 2 == 0) {
db.delete(records.get(i).getKey());
}
}
for (int i = 0; i < records.size(); i++) {
byte[] actual = db.get(records.get(i).getKey());
if (i % 2 == 0) {
Assert.assertNull(actual);
}
else {
Assert.assertEquals(records.get(i).getValue(), actual);
}
}
// insert deleted records.
for (int i = 0; i < records.size(); i++) {
if (i % 2 == 0) {
byte[] value = TestUtils.generateRandomByteArray();
byte[] key = records.get(i).getKey();
db.put(key, value);
records.set(i, new Record(key, value));
}
}
records.forEach(record -> {
try {
byte[] value = db.get(record.getKey());
Assert.assertEquals(record.getValue(), value);
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
});
// also check the iterator.
List actual = new ArrayList<>();
db.newIterator().forEachRemaining(actual::add);
Assert.assertTrue(actual.containsAll(records) && records.containsAll(actual));
}
@Test(dataProvider = "Options")
public void testDeleteAndOpen(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBDeletionTest", "testDeleteAndOpen");
options.setCompactionDisabled(true);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
// delete every other record.
for (int i = 0; i < records.size(); i++) {
if (i % 2 == 0) {
db.delete(records.get(i).getKey());
}
}
db.close();
db = getTestDBWithoutDeletingFiles(directory, options);
for (int i = 0; i < records.size(); i++) {
byte[] actual = db.get(records.get(i).getKey());
if (i % 2 == 0) {
Assert.assertNull(actual);
}
else {
Assert.assertEquals(records.get(i).getValue(), actual);
}
}
}
@Test(dataProvider = "Options")
public void testDeleteAndMerge(HaloDBOptions options) throws Exception {
String directory = TestUtils.getTestDirectory("HaloDBDeletionTest", "testDeleteAndMerge");
options.setMaxFileSize(10 * 1024);
options.setCompactionThresholdPerFile(0.10);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
// delete records
Random random = new Random();
Set deleted = new HashSet<>();
List newRecords = new ArrayList<>();
for (int i = 0; i < 1000; i++) {
int index = random.nextInt(records.size());
db.delete(records.get(index).getKey());
deleted.add(index);
// also throw in some new records into to mix.
// size is 40 so that we create keys distinct from
// what we used before.
byte[] key = TestUtils.generateRandomByteArray(40);
db.put(key, TestUtils.generateRandomByteArray());
newRecords.add(key);
}
// update the new records to make sure the the files containing tombstones
// will be compacted.
for (byte[] key : newRecords) {
db.put(key, TestUtils.generateRandomByteArray());
}
TestUtils.waitForCompactionToComplete(db);
db.close();
db = getTestDBWithoutDeletingFiles(directory, options);
for (int i = 0; i < records.size(); i++) {
byte[] actual = db.get(records.get(i).getKey());
if (deleted.contains(i)) {
Assert.assertNull(actual);
}
else {
Assert.assertEquals(records.get(i).getValue(), actual);
}
}
}
@Test(dataProvider = "Options")
public void testDeleteAllRecords(HaloDBOptions options) throws Exception {
String directory = TestUtils.getTestDirectory("HaloDBDeletionTest", "testDeleteAllRecords");
options.setMaxFileSize(10 * 1024);
options.setCompactionThresholdPerFile(1);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
// There will be 1000 files each of size 10KB
List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024 - Record.Header.HEADER_SIZE);
// delete all records.
for (Record r : records) {
db.delete(r.getKey());
}
TestUtils.waitForCompactionToComplete(db);
Assert.assertEquals(db.size(), 0);
for (Record r : records) {
Assert.assertNull(db.get(r.getKey()));
}
// only the current write file will be remaining everything else should have been
// deleted by the compaction job.
Assert.assertEquals(FileUtils.listDataFiles(new File(directory)).length, 1);
}
}
================================================
FILE: src/test/java/com/oath/halodb/HaloDBFileCompactionTest.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import com.google.common.primitives.Longs;
import org.hamcrest.MatcherAssert;
import org.hamcrest.Matchers;
import org.testng.annotations.Test;
import java.io.File;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
public class HaloDBFileCompactionTest extends TestBase {
@Test(dataProvider = "Options")
public void testCompaction(HaloDBOptions options) throws Exception {
String directory = TestUtils.getTestDirectory("HaloDBFileCompactionTest", "testCompaction");
int recordSize = 1024;
int recordNumber = 20;
options.setMaxFileSize(10 * recordSize); // 10 records per data file.
options.setCompactionThresholdPerFile(0.5);
HaloDB db = getTestDB(directory, options);
byte[] data = new byte[recordSize - Record.Header.HEADER_SIZE - 8 - 8];
for (int i = 0; i < data.length; i++) {
data[i] = (byte)i;
}
Record[] records = new Record[recordNumber];
for (int i = 0; i < recordNumber; i++) {
byte[] key = Longs.toByteArray(i);
byte[] value = TestUtils.concatenateArrays(data, key);
records[i] = new Record(key, value);
db.put(records[i].getKey(), records[i].getValue());
}
List freshRecords = new ArrayList<>();
// There are two data files. make the first half of both the files stale.
for (int i = 0; i < 5; i++) {
db.put(records[i].getKey(), records[i].getValue());
db.put(records[i+10].getKey(), records[i+10].getValue());
}
// Second half of both the files should be copied to the compacted file.
for (int i = 5; i < 10; i++) {
freshRecords.add(records[i]);
freshRecords.add(records[i + 10]);
}
TestUtils.waitForCompactionToComplete(db);
// the latest file will be the compacted file.
File compactedFile = Arrays.stream(FileUtils.listDataFiles(new File(directory))).max(Comparator.comparing(File::getName)).get();
HaloDBFile.HaloDBFileIterator iterator = HaloDBFile.openForReading(dbDirectory, compactedFile, HaloDBFile.FileType.COMPACTED_FILE, options).newIterator();
// make sure the the compacted file has the bottom half of two files.
List mergedRecords = new ArrayList<>();
while (iterator.hasNext()) {
mergedRecords.add(iterator.next());
}
MatcherAssert.assertThat(freshRecords, Matchers.containsInAnyOrder(mergedRecords.toArray()));
}
}
================================================
FILE: src/test/java/com/oath/halodb/HaloDBFileTest.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import org.testng.Assert;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.nio.file.Paths;
import java.nio.file.StandardOpenOption;
import java.nio.file.attribute.FileTime;
import java.util.List;
public class HaloDBFileTest {
private File directory = Paths.get("tmp", "HaloDBFileTest", "testIndexFile").toFile();
private DBDirectory dbDirectory;
private HaloDBFile file;
private IndexFile indexFile;
private int fileId = 100;
private File backingFile = directory.toPath().resolve(fileId+HaloDBFile.DATA_FILE_NAME).toFile();
private FileTime createdTime;
@BeforeMethod
public void before() throws IOException {
TestUtils.deleteDirectory(directory);
dbDirectory = DBDirectory.open(directory);
file = HaloDBFile.create(dbDirectory, fileId, new HaloDBOptions(), HaloDBFile.FileType.DATA_FILE);
createdTime = TestUtils.getFileCreationTime(backingFile);
indexFile = new IndexFile(fileId, dbDirectory, new HaloDBOptions());
try {
// wait for a second to make sure that the file creation time of the repaired file will be different.
Thread.sleep(1000);
} catch (InterruptedException e) {
}
}
@AfterMethod
public void after() throws IOException {
if (file != null)
file.close();
if (indexFile != null)
indexFile.close();
dbDirectory.close();
TestUtils.deleteDirectory(directory);
}
@Test
public void testIndexFile() throws IOException {
List list = insertTestRecords();
indexFile.open();
verifyIndexFile(indexFile, list);
}
@Test
public void testFileWithInvalidRecord() throws IOException {
List list = insertTestRecords();
// write a corrupted header to file.
try(FileChannel channel = FileChannel.open(Paths.get(directory.getCanonicalPath(), fileId + HaloDBFile.DATA_FILE_NAME).toAbsolutePath(), StandardOpenOption.APPEND)) {
ByteBuffer data = ByteBuffer.wrap("garbage".getBytes());
channel.write(data);
}
HaloDBFile.HaloDBFileIterator iterator = file.newIterator();
int count = 0;
while (iterator.hasNext() && count < 100) {
Record record = iterator.next();
Assert.assertEquals(record.getKey(), list.get(count++).getKey());
}
// 101th record's header is corrupted.
Assert.assertTrue(iterator.hasNext());
// Since header is corrupted we won't be able to read it and hence next will return null.
Assert.assertNull(iterator.next());
}
@Test
public void testCorruptedHeader() throws IOException {
List list = insertTestRecords();
// write a corrupted header to file.
// write a corrupted record to file.
byte[] key = "corrupted key".getBytes();
byte[] value = "corrupted value".getBytes();
Record corrupted = new Record(key, value);
// value length is corrupted.
corrupted.setHeader(new Record.Header(0, 0, (byte)key.length, -345445, 1234));
try(FileChannel channel = FileChannel.open(Paths.get(directory.getCanonicalPath(), fileId + HaloDBFile.DATA_FILE_NAME).toAbsolutePath(), StandardOpenOption.APPEND)) {
channel.write(corrupted.serialize());
}
HaloDBFile.HaloDBFileIterator iterator = file.newIterator();
int count = 0;
while (iterator.hasNext() && count < 100) {
Record r = iterator.next();
Assert.assertEquals(r.getKey(), list.get(count).getKey());
Assert.assertEquals(r.getValue(), list.get(count).getValue());
count++;
}
// 101th record's header is corrupted.
Assert.assertTrue(iterator.hasNext());
// Since header is corrupted we won't be able to read it and hence next will return null.
Assert.assertNull(iterator.next());
}
@Test
public void testRebuildIndexFile() throws IOException {
List list = insertTestRecords();
indexFile.delete();
// make sure that the file is deleted.
Assert.assertFalse(Paths.get(directory.getName(), fileId + IndexFile.INDEX_FILE_NAME).toFile().exists());
file.rebuildIndexFile();
indexFile.open();
verifyIndexFile(indexFile, list);
}
@Test
public void testRepairDataFileWithCorruptedValue() throws IOException {
List list = insertTestRecords();
// write a corrupted record to file.
// the record is corrupted in such a way the the size is unchanged but the contents have changed, thus crc will be different.
byte[] key = "corrupted key".getBytes();
byte[] value = "corrupted value".getBytes();
Record record = new Record(key, value);
record.setHeader(new Record.Header(0, 2, (byte)key.length, value.length, 1234));
try(FileChannel channel = FileChannel.open(Paths.get(directory.getCanonicalPath(), fileId + HaloDBFile.DATA_FILE_NAME).toAbsolutePath(), StandardOpenOption.APPEND)) {
ByteBuffer[] data = record.serialize();
data[2] = ByteBuffer.wrap("value corrupted".getBytes());
channel.write(data);
}
HaloDBFile repairedFile = file.repairFile(dbDirectory);
Assert.assertNotEquals(TestUtils.getFileCreationTime(backingFile), createdTime);
Assert.assertEquals(repairedFile.getPath(), file.getPath());
verifyDataFile(list, repairedFile);
verifyIndexFile(repairedFile.getIndexFile(), list);
}
@Test
public void testRepairDataFileWithInCompleteRecord() throws IOException {
List list = insertTestRecords();
// write a corrupted record to file.
// value was not completely written to file.
byte[] key = "corrupted key".getBytes();
byte[] value = "corrupted value".getBytes();
Record record = new Record(key, value);
record.setHeader(new Record.Header(0, 100, (byte)key.length, value.length, 1234));
try(FileChannel channel = FileChannel.open(Paths.get(directory.getCanonicalPath(), fileId + HaloDBFile.DATA_FILE_NAME).toAbsolutePath(), StandardOpenOption.APPEND)) {
ByteBuffer[] data = record.serialize();
data[2] = ByteBuffer.wrap("missing".getBytes());
channel.write(data);
}
HaloDBFile repairedFile = file.repairFile(dbDirectory);
Assert.assertNotEquals(TestUtils.getFileCreationTime(backingFile), createdTime);
Assert.assertEquals(repairedFile.getPath(), file.getPath());
verifyDataFile(list, repairedFile);
verifyIndexFile(repairedFile.getIndexFile(), list);
}
@Test
public void testRepairDataFileContainingRecordsWithCorruptedHeader() throws IOException {
List list = insertTestRecords();
// write a corrupted header to file.
try(FileChannel channel = FileChannel.open(Paths.get(directory.getCanonicalPath(), fileId + HaloDBFile.DATA_FILE_NAME).toAbsolutePath(), StandardOpenOption.APPEND)) {
ByteBuffer data = ByteBuffer.wrap("garbage".getBytes());
channel.write(data);
}
HaloDBFile repairedFile = file.repairFile(dbDirectory);
Assert.assertNotEquals(TestUtils.getFileCreationTime(backingFile), createdTime);
Assert.assertEquals(repairedFile.getPath(), file.getPath());
verifyDataFile(list, repairedFile);
verifyIndexFile(repairedFile.getIndexFile(), list);
}
@Test
public void testRepairDataFileContainingRecordsWithValidButCorruptedHeader() throws IOException {
List list = insertTestRecords();
// write a corrupted record to file.
byte[] key = "corrupted key".getBytes();
byte[] value = "corrupted value".getBytes();
Record record = new Record(key, value);
// header is valid but the value size is incorrect.
record.setHeader(new Record.Header(0,101, (byte)key.length, 5, 1234));
try(FileChannel channel = FileChannel.open(Paths.get(directory.getCanonicalPath(), fileId + HaloDBFile.DATA_FILE_NAME).toAbsolutePath(), StandardOpenOption.APPEND)) {
ByteBuffer[] data = record.serialize();
channel.write(data);
}
HaloDBFile repairedFile = file.repairFile(dbDirectory);
Assert.assertNotEquals(TestUtils.getFileCreationTime(backingFile), createdTime);
Assert.assertEquals(repairedFile.getPath(), file.getPath());
verifyDataFile(list, repairedFile);
verifyIndexFile(repairedFile.getIndexFile(), list);
}
private void verifyIndexFile(IndexFile file, List recordList) throws IOException {
IndexFile.IndexFileIterator indexFileIterator = file.newIterator();
int count = 0;
while (indexFileIterator.hasNext()) {
IndexFileEntry e = indexFileIterator.next();
Record r = recordList.get(count++);
InMemoryIndexMetaData meta = r.getRecordMetaData();
Assert.assertEquals(e.getKey(), r.getKey());
int expectedOffset = meta.getValueOffset() - Record.Header.HEADER_SIZE - r.getKey().length;
Assert.assertEquals(e.getRecordOffset(), expectedOffset);
}
Assert.assertEquals(count, recordList.size());
}
private List insertTestRecords() throws IOException {
List list = TestUtils.generateRandomData(100);
for (Record record : list) {
record.setSequenceNumber(100);
InMemoryIndexMetaData meta = file.writeRecord(record);
record.setRecordMetaData(meta);
}
return list;
}
private void verifyDataFile(List recordList, HaloDBFile dataFile) throws IOException {
HaloDBFile.HaloDBFileIterator iterator = dataFile.newIterator();
int count = 0;
while (iterator.hasNext()) {
Record actual = iterator.next();
Record expected = recordList.get(count++);
Assert.assertEquals(actual, expected);
}
Assert.assertEquals(count, recordList.size());
}
}
================================================
FILE: src/test/java/com/oath/halodb/HaloDBIteratorTest.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import org.hamcrest.MatcherAssert;
import org.hamcrest.Matchers;
import org.testng.Assert;
import org.testng.annotations.Test;
import java.io.IOException;
import java.nio.channels.ClosedChannelException;
import java.util.ArrayList;
import java.util.List;
import java.util.NoSuchElementException;
import mockit.Invocation;
import mockit.Mock;
import mockit.MockUp;
public class HaloDBIteratorTest extends TestBase {
@Test(expectedExceptions = NoSuchElementException.class, dataProvider = "Options")
public void testWithEmptyDB(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBIteratorTest", "testWithEmptyDB");
HaloDB db = getTestDB(directory, options);
HaloDBIterator iterator = db.newIterator();
Assert.assertFalse(iterator.hasNext());
iterator.next();
}
@Test(dataProvider = "Options")
public void testWithDelete(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBIteratorTest", "testWithEmptyDB");
options.setCompactionDisabled(true);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
// delete all records.
for (Record r : records) {
db.delete(r.getKey());
}
HaloDBIterator iterator = db.newIterator();
Assert.assertFalse(iterator.hasNext());
// close and open the db again.
db.close();
db = getTestDBWithoutDeletingFiles(directory, options);
iterator = db.newIterator();
Assert.assertFalse(iterator.hasNext());
}
@Test(dataProvider = "Options")
public void testPutAndGetDB(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBIteratorTest", "testPutAndGetDB");
options.setCompactionDisabled(true);
options.setMaxFileSize(10 * 1024);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
List actual = new ArrayList<>();
db.newIterator().forEachRemaining(actual::add);
MatcherAssert.assertThat(actual, Matchers.containsInAnyOrder(records.toArray()));
}
@Test(dataProvider = "Options")
public void testPutUpdateAndGetDB(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBIteratorTest", "testPutUpdateAndGetDB");
options.setCompactionDisabled(true);
options.setMaxFileSize(10 * 1024);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
List updated = TestUtils.updateRecords(db, records);
List actual = new ArrayList<>();
db.newIterator().forEachRemaining(actual::add);
MatcherAssert.assertThat(actual, Matchers.containsInAnyOrder(updated.toArray()));
}
@Test(dataProvider = "Options")
public void testPutUpdateCompactAndGetDB(HaloDBOptions options) throws HaloDBException, InterruptedException {
String directory = TestUtils.getTestDirectory("HaloDBIteratorTest", "testPutUpdateMergeAndGetDB");
options.setMaxFileSize(10 * 1024);
options.setCompactionThresholdPerFile(0.50);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
List updated = TestUtils.updateRecords(db, records);
TestUtils.waitForCompactionToComplete(db);
List actual = new ArrayList<>();
db.newIterator().forEachRemaining(actual::add);
Assert.assertEquals(actual.size(), updated.size());
MatcherAssert.assertThat(actual, Matchers.containsInAnyOrder(updated.toArray()));
}
// Test to make sure that no exceptions are thrown when files are being deleted by
// compaction thread and db is being iterated.
@Test(dataProvider = "Options")
public void testConcurrentCompactionAndIterator(HaloDBOptions options) throws HaloDBException, InterruptedException {
String directory = TestUtils.getTestDirectory("HaloDBIteratorTest", "testConcurrentCompactionAndIterator");
options.setMaxFileSize(1024 * 1024);
options.setCompactionThresholdPerFile(0.1);
final HaloDB db = getTestDB(directory, options);
// insert 1024 records per file, and a total of 10 files.
int noOfRecords = 10*1024;
List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-Record.Header.HEADER_SIZE);
int noOfUpdateRuns = 10;
Thread updateThread = new Thread(() -> {
for (int i=0; i() {
@Mock
byte[] readFromFile(Invocation invocation, int offset, int length) throws HaloDBException {
try {
// In the iterator after reading from keyCache pause for a while
// to increase the chance of file being closed by compaction thread.
Thread.sleep(100);
} catch (InterruptedException e) {
}
return invocation.proceed(offset, length);
}
};
options.setMaxFileSize(2 * 1024);
options.setCompactionThresholdPerFile(0.1);
final HaloDB db = getTestDB(directory, options);
int noOfRecords = 4; // 2 records on 2 files.
List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-Record.Header.HEADER_SIZE);
int noOfUpdateRuns = 1000;
Thread updateThread = new Thread(() -> {
for (int i=0; i() {
int count = 0;
@Mock
byte[] readFromFile(Invocation invocation, int offset, int length) throws IOException {
if (count % 3 == 0) {
throw new ClosedChannelException();
}
return invocation.proceed(offset, length);
}
};
options.setMaxFileSize(10 * 1024);
options.setCompactionThresholdPerFile(0.6);
final HaloDB db = getTestDB(directory, options);
int noOfRecords = 50; // 50 records on 5 files.
List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-Record.Header.HEADER_SIZE);
int noOfUpdateRuns = 100;
Thread updateThread = new Thread(() -> {
for (int i=0; i records = TestUtils.insertRandomRecords(db, noOfRecords);
// delete all records.
for (Record r : records) {
db.delete(r.getKey());
}
HaloDBKeyIterator iterator = db.newKeyIterator();
Assert.assertFalse(iterator.hasNext());
// close and open the db again.
db.close();
db = getTestDBWithoutDeletingFiles(directory, options);
iterator = db.newKeyIterator();
Assert.assertFalse(iterator.hasNext());
}
@Test(dataProvider = "Options")
public void testPutAndGetDB(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBKeyIteratorTest", "testPutAndGetDB");
options.setCompactionDisabled(true);
options.setMaxFileSize(10 * 1024);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
List keys = new LinkedList<>();
for (Record record : records) {
keys.add(new RecordKey(record.getKey()));
}
List actual = new ArrayList<>();
db.newKeyIterator().forEachRemaining(actual::add);
MatcherAssert.assertThat(actual, Matchers.containsInAnyOrder(keys.toArray()));
}
@Test(dataProvider = "Options")
public void testPutUpdateAndGetDB(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBKeyIteratorTest", "testPutUpdateAndGetDB");
options.setCompactionDisabled(true);
options.setMaxFileSize(10 * 1024);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
List updated = TestUtils.updateRecords(db, records);
List keys = new LinkedList<>();
for (Record record : updated) {
keys.add(new RecordKey(record.getKey()));
}
List actual = new ArrayList<>();
db.newKeyIterator().forEachRemaining(actual::add);
MatcherAssert.assertThat(actual, Matchers.containsInAnyOrder(keys.toArray()));
}
}
================================================
FILE: src/test/java/com/oath/halodb/HaloDBOptionsTest.java
================================================
package com.oath.halodb;
import org.testng.Assert;
import org.testng.annotations.Test;
import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException;
public class HaloDBOptionsTest extends TestBase {
@Test
public void testDefaultOptions() throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBOptionsTest", "testDefaultOptions");
HaloDB db = getTestDB(directory, new HaloDBOptions());
Assert.assertFalse(db.stats().getOptions().isSyncWrite());
Assert.assertFalse(db.stats().getOptions().isCompactionDisabled());
Assert.assertEquals(db.stats().getOptions().getBuildIndexThreads(), 1);
}
@Test
public void testSetBuildIndexThreads() {
int availableProcessors = Runtime.getRuntime().availableProcessors();
HaloDBOptions options = new HaloDBOptions();
// Test valid boundaries.
if (availableProcessors > 1) {
options.setBuildIndexThreads(availableProcessors);
Assert.assertEquals(options.getBuildIndexThreads(), availableProcessors);
}
options.setBuildIndexThreads(1);
Assert.assertEquals(options.getBuildIndexThreads(), 1);
// Test invalid boundaries.
assertThatIllegalArgumentException().isThrownBy(() -> options.setBuildIndexThreads(0));
assertThatIllegalArgumentException().isThrownBy(() -> options.setBuildIndexThreads(availableProcessors + 1));
}
}
================================================
FILE: src/test/java/com/oath/halodb/HaloDBStatsTest.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import org.testng.Assert;
import org.testng.annotations.Test;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
public class HaloDBStatsTest extends TestBase {
@Test(dataProvider = "Options")
public void testOptions(HaloDBOptions options) throws HaloDBException {
String dir = TestUtils.getTestDirectory("HaloDBStatsTest", "testOptions");
options.setMaxFileSize(10 * 1024);
options.setCompactionDisabled(true);
options.setCompactionThresholdPerFile(0.9);
options.setFlushDataSizeBytes(1024);
options.setCompactionJobRate(2048);
options.setNumberOfRecords(100);
options.setCleanUpInMemoryIndexOnClose(true);
HaloDB db = getTestDB(dir, options);
HaloDBStats stats = db.stats();
HaloDBOptions actual = stats.getOptions();
Assert.assertEquals(actual.getMaxFileSize(), options.getMaxFileSize());
Assert.assertEquals(actual.getCompactionThresholdPerFile(), options.getCompactionThresholdPerFile());
Assert.assertEquals(actual.getFlushDataSizeBytes(), options.getFlushDataSizeBytes());
Assert.assertEquals(actual.getCompactionJobRate(), options.getCompactionJobRate());
Assert.assertEquals(actual.getNumberOfRecords(), options.getNumberOfRecords());
Assert.assertEquals(actual.isCleanUpInMemoryIndexOnClose(), options.isCleanUpInMemoryIndexOnClose());
Assert.assertEquals(stats.getSize(), 0);
}
@Test(dataProvider = "Options")
public void testStaleMap(HaloDBOptions options) throws HaloDBException {
String dir = TestUtils.getTestDirectory("HaloDBStatsTest", "testStaleMap");
options.setMaxFileSize(10 * 1024);
options.setCompactionThresholdPerFile(0.50);
HaloDB db = getTestDB(dir, options);
// will create 10 files with 10 records each.
int recordSize = 1024 - Record.Header.HEADER_SIZE;
List records = TestUtils.insertRandomRecordsOfSize(db, 100, recordSize);
// No updates hence stale data map should be empty.
Assert.assertEquals(db.stats().getStaleDataPercentPerFile().size(), 0);
for (int i = 0; i < records.size(); i++) {
if (i % 10 == 0)
db.put(records.get(i).getKey(), TestUtils.generateRandomByteArray(recordSize));
}
// Updated 1 out of 10 records in each file, hence 10% stale data.
Assert.assertEquals(db.stats().getStaleDataPercentPerFile().size(), 10);
db.stats().getStaleDataPercentPerFile().forEach((k, v) -> {
Assert.assertEquals(v, 10.0);
});
Assert.assertEquals(db.stats().getSize(), 100);
}
@Test(dataProvider = "Options")
public void testCompactionStats(HaloDBOptions options) throws HaloDBException {
String dir = TestUtils.getTestDirectory("HaloDBStatsTest", "testCompactionStats");
options.setMaxFileSize(10 * 1024);
options.setCompactionThresholdPerFile(0.50);
options.setCompactionDisabled(true);
HaloDB db = getTestDB(dir, options);
// will create 10 files with 10 records each.
int recordSize = 1024 - Record.Header.HEADER_SIZE;
int noOfRecords = 100;
List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, recordSize);
Assert.assertEquals(db.stats().getNumberOfDataFiles(), 10);
Assert.assertEquals(db.stats().getNumberOfTombstoneFiles(), 0);
// update 50% of records in each file.
for (int i = 0; i < records.size(); i++) {
if (i % 10 < 5)
db.put(records.get(i).getKey(), TestUtils.generateRandomByteArray(records.get(i).getValue().length));
}
TestUtils.waitForCompactionToComplete(db);
// compaction stats are 0 since compaction is paused.
Assert.assertFalse(db.stats().isCompactionRunning());
Assert.assertEquals(db.stats().getCompactionRateInInternal(), 0);
Assert.assertEquals(db.stats().getCompactionRateSinceBeginning(), 0);
Assert.assertNotEquals(db.stats().toString().length(), 0);
Assert.assertEquals(db.stats().getNumberOfDataFiles(), 15);
Assert.assertEquals(db.stats().getNumberOfTombstoneFiles(), 0);
db.close();
options.setCompactionDisabled(false);
db = getTestDBWithoutDeletingFiles(dir, options);
Assert.assertTrue(db.stats().isCompactionRunning());
TestUtils.waitForCompactionToComplete(db);
// compaction complete hence stale data map is empty.
HaloDBStats stats = db.stats();
Assert.assertEquals(stats.getStaleDataPercentPerFile().size(), 0);
Assert.assertEquals(stats.getNumberOfFilesPendingCompaction(), 0);
Assert.assertEquals(stats.getNumberOfRecordsCopied(), noOfRecords / 2);
Assert.assertEquals(stats.getNumberOfRecordsReplaced(), noOfRecords / 2);
Assert.assertEquals(stats.getNumberOfRecordsScanned(), noOfRecords);
Assert.assertEquals(stats.getSizeOfRecordsCopied(), noOfRecords / 2 * 1024);
Assert.assertEquals(stats.getSizeOfFilesDeleted(), options.getMaxFileSize() * 10);
Assert.assertEquals(stats.getSizeReclaimed(), options.getMaxFileSize() * 10 / 2);
Assert.assertEquals(stats.getSize(), noOfRecords);
Assert.assertNotEquals(db.stats().getCompactionRateInInternal(), 0);
Assert.assertNotEquals(db.stats().getCompactionRateSinceBeginning(), 0);
Assert.assertNotEquals(db.stats().toString().length(), 0);
Assert.assertEquals(db.stats().getNumberOfDataFiles(), 10);
Assert.assertEquals(db.stats().getNumberOfTombstoneFiles(), 0);
// delete the 50% of records
for (int i = 0; i < records.size(); i++) {
if (i % 10 < 5)
db.delete(records.get(i).getKey());
}
// restart compaction thread because it was stopped by TestUtils.waitForCompactionToComplete(db)
db.resumeCompaction();
TestUtils.waitForCompactionToComplete(db);
Assert.assertEquals(db.stats().getNumberOfDataFiles(), 5);
Assert.assertEquals(db.stats().getNumberOfTombstoneFiles(), 1);
db.close();
// all delete records have been removed by compaction job, so no record copied at reopen
options.setCleanUpTombstonesDuringOpen(true);
db = getTestDBWithoutDeletingFiles(dir, options);
stats = db.stats();
Assert.assertEquals(stats.getSize(), noOfRecords / 2);
Assert.assertEquals(db.stats().getNumberOfTombstonesCleanedUpDuringOpen(), noOfRecords / 2);
Assert.assertEquals(db.stats().getNumberOfTombstonesFoundDuringOpen(), noOfRecords / 2);
Assert.assertEquals(db.stats().getNumberOfDataFiles(), 5);
Assert.assertEquals(db.stats().getNumberOfTombstoneFiles(), 0);
db.resetStats();
stats = db.stats();
Assert.assertEquals(stats.getNumberOfFilesPendingCompaction(), 0);
Assert.assertEquals(stats.getNumberOfRecordsCopied(), 0);
Assert.assertEquals(stats.getNumberOfRecordsReplaced(), 0);
Assert.assertEquals(stats.getNumberOfRecordsScanned(), 0);
Assert.assertEquals(stats.getSizeOfRecordsCopied(), 0);
Assert.assertEquals(stats.getSizeOfFilesDeleted(), 0);
Assert.assertEquals(stats.getSizeReclaimed(), 0);
Assert.assertEquals(stats.getSize(), noOfRecords / 2);
Assert.assertEquals(db.stats().getCompactionRateInInternal(), 0);
Assert.assertNotEquals(db.stats().getCompactionRateSinceBeginning(), 0);
Assert.assertNotEquals(db.stats().toString().length(), 0);
}
@Test(dataProvider = "Options")
public void testIndexStats(HaloDBOptions options) throws HaloDBException {
String dir = TestUtils.getTestDirectory("HaloDBStatsTest", "testIndexStats");
int numberOfSegments = (int)Utils.roundUpToPowerOf2(Runtime.getRuntime().availableProcessors() * 2);
int numberOfRecords = numberOfSegments * 1024;
options.setMaxFileSize(10 * 1024);
options.setNumberOfRecords(numberOfRecords);
options.setCompactionThresholdPerFile(0.50);
options.setCompactionDisabled(true);
HaloDB db = getTestDB(dir, options);
HaloDBStats stats = db.stats();
Assert.assertEquals(numberOfSegments, stats.getNumberOfSegments());
Assert.assertEquals(numberOfRecords/numberOfSegments, stats.getMaxSizePerSegment());
SegmentStats[] expected = new SegmentStats[numberOfSegments];
SegmentStats s;
if (options.isUseMemoryPool()) {
s = new SegmentStats(0, 0, 0, 0);
}
else {
s = new SegmentStats(0, -1, -1, -1);
}
Arrays.fill(expected, s);
Assert.assertEquals(stats.getSegmentStats(), expected);
}
@Test(dataProvider = "Options")
public void testStatsToStringMap(HaloDBOptions options) throws HaloDBException {
String dir = TestUtils.getTestDirectory("HaloDBStatsTest", "testStatsToStringMap");
HaloDB db = getTestDB(dir, options);
HaloDBStats stats = db.stats();
Map map = stats.toStringMap();
Assert.assertEquals(map.size(), 22);
Assert.assertNotNull(map.get("statsResetTime"));
Assert.assertNotNull(map.get("size"));
Assert.assertNotNull(map.get("Options"));
Assert.assertNotNull(map.get("isCompactionRunning"));
Assert.assertNotNull(map.get("CompactionJobRateInInterval"));
Assert.assertNotNull(map.get("CompactionJobRateSinceBeginning"));
Assert.assertNotNull(map.get("numberOfFilesPendingCompaction"));
Assert.assertNotNull(map.get("numberOfRecordsCopied"));
Assert.assertNotNull(map.get("numberOfRecordsReplaced"));
Assert.assertNotNull(map.get("numberOfRecordsScanned"));
Assert.assertNotNull(map.get("sizeOfRecordsCopied"));
Assert.assertNotNull(map.get("sizeOfFilesDeleted"));
Assert.assertNotNull(map.get("sizeReclaimed"));
Assert.assertNotNull(map.get("rehashCount"));
Assert.assertNotNull(map.get("maxSizePerSegment"));
Assert.assertNotNull(map.get("numberOfDataFiles"));
Assert.assertNotNull(map.get("numberOfTombstoneFiles"));
Assert.assertNotNull(map.get("numberOfTombstonesFoundDuringOpen"));
Assert.assertNotNull(map.get("numberOfTombstonesCleanedUpDuringOpen"));
Assert.assertNotNull(map.get("segmentStats"));
Assert.assertNotNull(map.get("numberOfSegments"));
Assert.assertNotNull(map.get("staleDataPercentPerFile"));
}
}
================================================
FILE: src/test/java/com/oath/halodb/HaloDBTest.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
package com.oath.halodb;
import com.google.common.primitives.Longs;
import java.io.File;
import java.nio.ByteBuffer;
import java.util.*;
import org.testng.Assert;
import org.testng.annotations.Test;
import java.io.IOException;
import java.nio.file.Paths;
import mockit.Mock;
import mockit.MockUp;
public class HaloDBTest extends TestBase {
@Test(dataProvider = "Options")
public void testPutAndGetDB(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testPutAndGetDB");
options.setCompactionDisabled(true);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
List actual = new ArrayList<>();
db.newIterator().forEachRemaining(actual::add);
Assert.assertTrue(actual.containsAll(records) && records.containsAll(actual));
records.forEach(record -> {
try {
byte[] value = db.get(record.getKey());
Assert.assertEquals(record.getValue(), value);
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
});
}
@Test(dataProvider = "Options")
public void testPutUpdateAndGetDB(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testPutUpdateAndGetDB");
options.setCompactionDisabled(true);
options.setMaxFileSize(10 * 1024);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
List updated = TestUtils.updateRecords(db, records);
List actual = new ArrayList<>();
db.newIterator().forEachRemaining(actual::add);
Assert.assertTrue(actual.containsAll(updated) && updated.containsAll(actual));
updated.forEach(record -> {
try {
byte[] value = db.get(record.getKey());
Assert.assertEquals(record.getValue(), value);
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
});
}
@Test(dataProvider = "Options")
public void testCreateCloseAndOpenDB(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testCreateCloseAndOpenDB");
options.setCompactionDisabled(true);
options.setMaxFileSize(10 * 1024);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
// update half the records.
for (int i = 0; i < records.size(); i++) {
if (i % 2 == 0) {
Record record = records.get(i);
try {
byte[] value = TestUtils.generateRandomByteArray();
db.put(record.getKey(), value);
records.set(i, new Record(record.getKey(), value));
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
}
}
db.close();
// open and read contents again.
HaloDB openAgainDB = getTestDBWithoutDeletingFiles(directory, options);
List actual = new ArrayList<>();
openAgainDB.newIterator().forEachRemaining(actual::add);
Assert.assertTrue(actual.containsAll(records) && records.containsAll(actual));
records.forEach(record -> {
try {
byte[] value = openAgainDB.get(record.getKey());
Assert.assertEquals(record.getValue(), value);
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
});
}
@Test(dataProvider = "Options")
public void testSyncWrite(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testSyncWrite");
options.setCompactionDisabled(true);
options.setMaxFileSize(10 * 1024);
options.enableSyncWrites(true);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
List updated = TestUtils.updateRecords(db, records);
List actual = new ArrayList<>();
db.newIterator().forEachRemaining(actual::add);
Assert.assertTrue(actual.containsAll(updated) && updated.containsAll(actual));
updated.forEach(record -> {
try {
byte[] value = db.get(record.getKey());
Assert.assertEquals(record.getValue(), value);
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
});
}
@Test(dataProvider = "Options")
public void testToCheckThatLatestUpdateIsPickedAfterDBOpen(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testToCheckThatLatestUpdateIsPickedAfterDBOpen");
options.setCompactionDisabled(true);
// sized to ensure that there will be two files.
options.setMaxFileSize(1500);
HaloDB db = getTestDB(directory, options);
byte[] key = TestUtils.generateRandomByteArray(7);
byte[] value = null;
// update the same record 100 times.
// each key-value pair with the metadata is 20 bytes.
// therefore 20 * 100 = 2000 bytes
for (int i = 0; i < 100; i++) {
value = TestUtils.generateRandomByteArray(7);
db.put(key, value);
}
db.close();
// open and read contents again.
HaloDB openAgainDB = getTestDBWithoutDeletingFiles(directory, options);
List actual = new ArrayList<>();
openAgainDB.newIterator().forEachRemaining(actual::add);
Assert.assertTrue(actual.size() == 1);
Assert.assertEquals(openAgainDB.get(key), value);
}
@Test(dataProvider = "Options")
public void testToCheckDelete(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testToCheckDelete");
options.setCompactionDisabled(true);
options.setMaxFileSize(10 * 1024);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
List deleted = new ArrayList<>();
for (int i = 0; i < noOfRecords; i++) {
if (i % 10 == 0) deleted.add(records.get(i));
}
TestUtils.deleteRecords(db, deleted);
List remaining = new ArrayList<>();
db.newIterator().forEachRemaining(remaining::add);
Assert.assertTrue(remaining.size() == noOfRecords - deleted.size());
deleted.forEach(r -> {
try {
Assert.assertNull(db.get(r.getKey()));
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
});
}
@Test(dataProvider = "Options")
public void testDeleteCloseAndOpen(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testDeleteCloseAndOpen");
options.setCompactionDisabled(true);
options.setMaxFileSize(10 * 1024);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
List deleted = new ArrayList<>();
for (int i = 0; i < noOfRecords; i++) {
if (i % 10 == 0) deleted.add(records.get(i));
}
TestUtils.deleteRecords(db, deleted);
db.close();
HaloDB openAgainDB = getTestDBWithoutDeletingFiles(directory, options);
List remaining = new ArrayList<>();
openAgainDB.newIterator().forEachRemaining(remaining::add);
Assert.assertTrue(remaining.size() == noOfRecords - deleted.size());
deleted.forEach(r -> {
try {
Assert.assertNull(openAgainDB.get(r.getKey()));
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
});
}
@Test(dataProvider = "Options")
public void testDeleteAndInsert(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testDeleteAndInsert");
options.setCompactionDisabled(true);
options.setMaxFileSize(10 * 1024);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
List deleted = new ArrayList<>();
for (int i = 0; i < noOfRecords; i++) {
if (i % 10 == 0) deleted.add(records.get(i));
}
TestUtils.deleteRecords(db, deleted);
List deleteAndInsert = new ArrayList<>();
deleted.forEach(r -> {
try {
byte[] value = TestUtils.generateRandomByteArray();
db.put(r.getKey(), value);
deleteAndInsert.add(new Record(r.getKey(), value));
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
});
List remaining = new ArrayList<>();
db.newIterator().forEachRemaining(remaining::add);
Assert.assertTrue(remaining.size() == noOfRecords);
deleteAndInsert.forEach(r -> {
try {
Assert.assertEquals(r.getValue(), db.get(r.getKey()));
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
});
}
@Test(dataProvider = "Options")
public void testDeleteInsertCloseAndOpen(HaloDBOptions options) throws HaloDBException {
String directory = TestUtils.getTestDirectory("tmp", "testDeleteInsertCloseAndOpen");
options.setCompactionDisabled(true);
options.setMaxFileSize(10 * 1024);
HaloDB db = getTestDB(directory, options);
int noOfRecords = 10_000;
List records = TestUtils.insertRandomRecords(db, noOfRecords);
List deleted = new ArrayList<>();
for (int i = 0; i < noOfRecords; i++) {
if (i % 10 == 0) deleted.add(records.get(i));
}
TestUtils.deleteRecords(db, deleted);
// inserting deleted records again.
List deleteAndInsert = new ArrayList<>();
deleted.forEach(r -> {
try {
byte[] value = TestUtils.generateRandomByteArray();
db.put(r.getKey(), value);
deleteAndInsert.add(new Record(r.getKey(), value));
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
});
db.close();
HaloDB openAgainDB = getTestDBWithoutDeletingFiles(directory, options);
List remaining = new ArrayList<>();
openAgainDB.newIterator().forEachRemaining(remaining::add);
Assert.assertTrue(remaining.size() == noOfRecords);
// make sure that records that were earlier deleted shows up now, since they were put back later.
deleteAndInsert.forEach(r -> {
try {
Assert.assertEquals(r.getValue(), openAgainDB.get(r.getKey()));
} catch (HaloDBException e) {
throw new RuntimeException(e);
}
});
}
@Test
public void testDBMetaFile() throws HaloDBException, IOException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testDBMetaFile");
HaloDBOptions options = new HaloDBOptions();
int maxFileSize = 1024 * 1024 * 1024;
options.setMaxFileSize(maxFileSize);
HaloDB db = getTestDB(directory, options);
// Make sure that the META file was written.
Assert.assertTrue(Paths.get(directory, DBMetaData.METADATA_FILE_NAME).toFile().exists());
DBMetaData metaData = new DBMetaData(dbDirectory);
metaData.loadFromFileIfExists();
// Make sure that the open flag was set on db open.
Assert.assertTrue(metaData.isOpen());
// Default value of ioError flag must be false.
Assert.assertFalse(metaData.isIOError());
// since we just created the db max file size should be set to one we set in HaloDBOptions
Assert.assertEquals(metaData.getMaxFileSize(), maxFileSize);
Assert.assertEquals(metaData.getVersion(), Versions.CURRENT_META_FILE_VERSION);
db.close();
// Make sure that the META file was written.
Assert.assertTrue(Paths.get(directory, DBMetaData.METADATA_FILE_NAME).toFile().exists());
// Make sure that the flags were set correctly on close.
metaData.loadFromFileIfExists();
Assert.assertEquals(metaData.getVersion(), Versions.CURRENT_META_FILE_VERSION);
Assert.assertFalse(metaData.isOpen());
Assert.assertFalse(metaData.isIOError());
Assert.assertEquals(metaData.getMaxFileSize(), maxFileSize);
}
@Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "File size cannot be changed after db was created.*")
public void testMaxFileSize() throws HaloDBException, IOException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testMaxFileSize");
HaloDBOptions options = new HaloDBOptions();
int maxFileSize = 1024 * 1024 * 1024;
options.setMaxFileSize(maxFileSize);
HaloDB db = getTestDB(directory, options);
DBMetaData metaData = new DBMetaData(dbDirectory);
metaData.loadFromFileIfExists();
Assert.assertEquals(metaData.getMaxFileSize(), maxFileSize);
db.close();
// try opening the db with max file size changed.
options.setMaxFileSize(500 * 1024 * 1024);
getTestDBWithoutDeletingFiles(directory, options);
}
@Test(expectedExceptions = HaloDBException.class, expectedExceptionsMessageRegExp = "Another process already holds a lock for this db.")
public void testLock() throws Throwable {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testLock");
HaloDB db = getTestDB(directory, new HaloDBOptions());
db.resetStats();
HaloDB anotherDB = HaloDB.open(directory, new HaloDBOptions());
anotherDB.resetStats();
}
@Test
public void testLockReleaseOnError() throws Throwable {
new MockUp() {
int count = 0;
@Mock
void loadFromFileIfExists() throws IOException {
System.out.println("Mock called");
if (count == 0) {
// throw an exception the first time the method is called.
count = 1;
throw new IOException();
}
}
};
String directory = TestUtils.getTestDirectory("HaloDBTest", "testLockReleaseOnError");
HaloDB db = null;
try {
db = getTestDB(directory, new HaloDBOptions());
} catch (HaloDBException e) {
// swallow the mocked exception.
}
// make sure open() failed.
Assert.assertNull(db);
// the lock should have been released when previous open() failed.
HaloDB anotherDB = getTestDBWithoutDeletingFiles(directory, new HaloDBOptions());
anotherDB.put(Longs.toByteArray(1), Longs.toByteArray(1));
Assert.assertEquals(anotherDB.size(), 1);
}
@Test(expectedExceptions = HaloDBException.class)
public void testPutAfterClose() throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testPutAfterClose");
HaloDB db = getTestDB(directory, new HaloDBOptions());
db.put(Longs.toByteArray(1), Longs.toByteArray(1));
db.close();
db.put(Longs.toByteArray(2), Longs.toByteArray(2));
}
@Test(expectedExceptions = HaloDBException.class)
public void testDeleteAfterClose() throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testDeleteAfterClose");
HaloDB db = getTestDB(directory, new HaloDBOptions());
db.put(Longs.toByteArray(1), Longs.toByteArray(1));
db.put(Longs.toByteArray(2), Longs.toByteArray(2));
db.delete(Longs.toByteArray(1));
db.close();
db.delete(Longs.toByteArray(2));
}
@Test(expectedExceptions = NullPointerException.class)
public void testPutAfterCloseWithoutWrites() throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testPutAfterCloseWithoutWrites");
HaloDB db = getTestDB(directory, new HaloDBOptions());
db.close();
db.put(Longs.toByteArray(1), Longs.toByteArray(1));
}
@Test(expectedExceptions = NullPointerException.class)
public void testDeleteAfterCloseWithoutWrites() throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testDeleteAfterCloseWithoutWrites");
HaloDB db = getTestDB(directory, new HaloDBOptions());
db.put(Longs.toByteArray(1), Longs.toByteArray(1));
Assert.assertEquals(db.get(Longs.toByteArray(1)), Longs.toByteArray(1));
db.close();
db.delete(Longs.toByteArray(1));
}
@Test
public void testSnapshot() throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testSnapshot");
HaloDBOptions options = new HaloDBOptions();
// Generate several data files
options.setMaxFileSize(10000);
HaloDB db = getTestDB(directory, options);
for (int i = 10000; i < 10000 + 10*1000; i++) {
db.put(ByteBuffer.allocate(4).putInt(i).array(), ByteBuffer.allocate(8).putInt(i).putInt(i*1024).array());
}
db.snapshot();
String snapshotDir = db.getSnapshotDirectory().toString();
HaloDB snapshotDB = getTestDBWithoutDeletingFiles(snapshotDir, options);
for (int i = 10000; i < 10000 + 10*1000; i++) {
byte[] value = snapshotDB.get(ByteBuffer.allocate(4).putInt(i).array());
Assert.assertTrue(Arrays.equals(value, ByteBuffer.allocate(8).putInt(i).putInt(i*1024).array()));
}
snapshotDB.close();
db.close();
}
@Test
public void testCreateAndDeleteSnapshot() throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testCreateAndDeleteSnapshot");
HaloDBOptions options = new HaloDBOptions();
// Generate several data files
options.setMaxFileSize(10000);
HaloDB db = getTestDB(directory, options);
for (int i = 10000; i < 10000 + 10*1000; i++) {
db.put(ByteBuffer.allocate(4).putInt(i).array(), ByteBuffer.allocate(8).putInt(i).putInt(i*1024).array());
}
Assert.assertTrue(db.clearSnapshot());
db.snapshot();
Assert.assertTrue(db.clearSnapshot());
File snapshotDir = db.getSnapshotDirectory();
Assert.assertFalse(snapshotDir.exists());
db.close();
}
@Test
public void testSnapshotAfterBeenCompacted() throws HaloDBException {
String directory = TestUtils.getTestDirectory("HaloDBTest", "testSnapshotAfterBeenCompacted");
HaloDBOptions options = new HaloDBOptions();
// Generate several data files
options.setMaxFileSize(10000);
options.setCompactionThresholdPerFile(0.7);
HaloDB db = getTestDB(directory, options);
for (int i = 10000; i < 10000 + 10*1000; i++) {
db.put(ByteBuffer.allocate(4).putInt(i).array(), ByteBuffer.allocate(8).putInt(i).putInt(i*1024).array());
}
db.snapshot();
// overwrite all previous record
for (int i = 10000; i < 10000 + 10*1000; i++) {
db.put(ByteBuffer.allocate(4).putInt(i).array(), ByteBuffer.allocate(8).putInt(i).putInt(i*2048).array());
}
TestUtils.waitForCompactionToComplete(db);
String snapshotDir = db.getSnapshotDirectory().toString();
HaloDB snapshotDB = getTestDBWithoutDeletingFiles(snapshotDir, options);
for (int i = 10000; i < 10000 + 10*1000; i++) {
byte[] key = ByteBuffer.allocate(4).putInt(i).array();
byte[] value = snapshotDB.get(key);
Assert.assertTrue(Arrays.equals(value, ByteBuffer.allocate(8).putInt(i).putInt(i*1024).array()));
Assert.assertFalse(Arrays.equals(value, db.get(key)));
}
snapshotDB.close();
db.close();
}
}
================================================
FILE: src/test/java/com/oath/halodb/HashTableTestUtils.java
================================================
/*
* Copyright 2018, Oath Inc
* Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms.
*/
// This code is a derivative work heavily modified from the OHC project. See NOTICE file for copyright and license.
package com.oath.halodb;
import com.google.common.base.Charsets;
import com.google.common.primitives.Longs;
import org.testng.Assert;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
final class HashTableTestUtils
{
public static final long ONE_MB = 1024 * 1024;
public static final HashTableValueSerializer stringSerializer = new HashTableValueSerializer()
{
public void serialize(String s, ByteBuffer buf)
{
byte[] bytes = s.getBytes(Charsets.UTF_8);
buf.put((byte) ((bytes.length >>> 8) & 0xFF));
buf.put((byte) ((bytes.length >>> 0) & 0xFF));
buf.put(bytes);
}
public String deserialize(ByteBuffer buf)
{
int length = (((buf.get() & 0xff) << 8) + ((buf.get() & 0xff) << 0));
byte[] bytes = new byte[length];
buf.get(bytes);
return new String(bytes, Charsets.UTF_8);
}
public int serializedSize(String s)
{
return writeUTFLen(s);
}
};
public static final HashTableValueSerializer byteArraySerializer = new HashTableValueSerializer()
{
@Override
public void serialize(byte[] value, ByteBuffer buf) {
buf.put(value);
}
@Override
public byte[] deserialize(ByteBuffer buf) {
// Cannot use buf.array() as buf is read-only for get() operations.
byte[] array = new byte[buf.remaining()];
buf.get(array);
return array;
}
@Override
public int serializedSize(byte[] value) {
return value.length;
}
};
public static final HashTableValueSerializer stringSerializerFailSerialize = new HashTableValueSerializer()
{
public void serialize(String s, ByteBuffer buf)
{
throw new RuntimeException("foo bar");
}
public String deserialize(ByteBuffer buf)
{
int length = (buf.get() << 8) + (buf.get() << 0);
byte[] bytes = new byte[length];
buf.get(bytes);
return new String(bytes, Charsets.UTF_8);
}
public int serializedSize(String s)
{
return writeUTFLen(s);
}
};
public static final HashTableValueSerializer stringSerializerFailDeserialize = new HashTableValueSerializer()
{
public void serialize(String s, ByteBuffer buf)
{
byte[] bytes = s.getBytes(Charsets.UTF_8);
buf.put((byte) ((bytes.length >>> 8) & 0xFF));
buf.put((byte) ((bytes.length >>> 0) & 0xFF));
buf.put(bytes);
}
public String deserialize(ByteBuffer buf)
{
throw new RuntimeException("foo bar");
}
public int serializedSize(String s)
{
return writeUTFLen(s);
}
};
public static final HashTableValueSerializer byteArraySerializerFailSerialize = new HashTableValueSerializer()
{
public void serialize(byte[] s, ByteBuffer buf)
{
throw new RuntimeException("foo bar");
}
public byte[] deserialize(ByteBuffer buf)
{
byte[] array = new byte[buf.remaining()];
buf.get(array);
return array;
}
public int serializedSize(byte[] s)
{
return s.length;
}
};
static int writeUTFLen(String str)
{
int strlen = str.length();
int utflen = 0;
int c;
for (int i = 0; i < strlen; i++)
{
c = str.charAt(i);
if ((c >= 0x0001) && (c <= 0x007F))
utflen++;
else if (c > 0x07FF)
utflen += 3;
else
utflen += 2;
}
if (utflen > 65535)
throw new RuntimeException("encoded string too long: " + utflen + " bytes");
return utflen + 2;
}
public static final byte[] dummyByteArray;
public static final HashTableValueSerializer intSerializer = new HashTableValueSerializer()
{
public void serialize(Integer s, ByteBuffer buf)
{
buf.put((byte)(1 & 0xff));
buf.putChar('A');
buf.putDouble(42.42424242d);
buf.putFloat(11.111f);
buf.putInt(s);
buf.putLong(Long.MAX_VALUE);
buf.putShort((short)(0x7654 & 0xFFFF));
buf.put(dummyByteArray);
}
public Integer deserialize(ByteBuffer buf)
{
Assert.assertEquals(buf.get(), (byte) 1);
Assert.assertEquals(buf.getChar(), 'A');
Assert.assertEquals(buf.getDouble(), 42.42424242d);
Assert.assertEquals(buf.getFloat(), 11.111f);
int r = buf.getInt();
Assert.assertEquals(buf.getLong(), Long.MAX_VALUE);
Assert.assertEquals(buf.getShort(), 0x7654);
byte[] b = new byte[dummyByteArray.length];
buf.get(b);
Assert.assertEquals(b, dummyByteArray);
return r;
}
public int serializedSize(Integer s)
{
return 529;
}
};
public static final HashTableValueSerializer