diff --git a/src/main/java/com/oath/halodb/CompactionManager.java b/src/main/java/com/oath/halodb/CompactionManager.java index 57ab5be..2776c02 100644 --- a/src/main/java/com/oath/halodb/CompactionManager.java +++ b/src/main/java/com/oath/halodb/CompactionManager.java @@ -5,18 +5,18 @@ 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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.RateLimiter; + class CompactionManager { private static final Logger logger = LoggerFactory.getLogger(CompactionManager.class); @@ -266,11 +266,8 @@ private void copyFreshRecordsToNewFile(int idOfFileToCompact) throws IOException ); currentWriteFile.getIndexFile().write(newEntry); - int valueOffset = Utils.getValueOffset(currentWriteFileOffset, key); - InMemoryIndexMetaData newMetaData = new InMemoryIndexMetaData( - currentWriteFile.getFileId(), valueOffset, - currentRecordMetaData.getValueSize(), indexFileEntry.getSequenceNumber() - ); + InMemoryIndexMetaData newMetaData = currentRecordMetaData + .relocated(currentWriteFile.getFileId(), currentWriteFileOffset); boolean updated = dbInternal.getInMemoryIndex().replace(key, currentRecordMetaData, newMetaData); if (updated) { @@ -302,7 +299,7 @@ private void copyFreshRecordsToNewFile(int idOfFileToCompact) throws IOException private boolean isRecordFresh(IndexFileEntry entry, InMemoryIndexMetaData metaData, int idOfFileToMerge) { return metaData != null && metaData.getFileId() == idOfFileToMerge - && metaData.getValueOffset() == Utils.getValueOffset(entry.getRecordOffset(), entry.getKey()); + && metaData.getValueOffset() == RecordEntry.getValueOffset(entry.getRecordOffset(), entry.getKey().length); } private void rollOverCurrentWriteFile(int recordSize) throws IOException { diff --git a/src/main/java/com/oath/halodb/FileUtils.java b/src/main/java/com/oath/halodb/FileUtils.java index 4d79f6d..725a430 100644 --- a/src/main/java/com/oath/halodb/FileUtils.java +++ b/src/main/java/com/oath/halodb/FileUtils.java @@ -7,10 +7,7 @@ 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; diff --git a/src/main/java/com/oath/halodb/HaloDB.java b/src/main/java/com/oath/halodb/HaloDB.java index c16d304..411ed98 100644 --- a/src/main/java/com/oath/halodb/HaloDB.java +++ b/src/main/java/com/oath/halodb/HaloDB.java @@ -5,11 +5,10 @@ package com.oath.halodb; -import com.google.common.annotations.VisibleForTesting; - import java.io.File; import java.io.IOException; -import java.util.Set; + +import com.google.common.annotations.VisibleForTesting; public final class HaloDB { diff --git a/src/main/java/com/oath/halodb/HaloDBFile.java b/src/main/java/com/oath/halodb/HaloDBFile.java index 89ceeb5..d72bc37 100644 --- a/src/main/java/com/oath/halodb/HaloDBFile.java +++ b/src/main/java/com/oath/halodb/HaloDBFile.java @@ -5,10 +5,8 @@ package com.oath.halodb; -import com.google.common.primitives.Ints; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; +import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; import java.io.File; import java.io.IOException; @@ -21,8 +19,10 @@ 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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.primitives.Ints; /** * Represents a data file and its associated index file. @@ -32,7 +32,8 @@ class HaloDBFile { private volatile int writeOffset; - private FileChannel channel; + private final FileChannel channel; + private final RandomAccessFile raf; private final File backingFile; private final DBDirectory dbDirectory; private final int fileId; @@ -49,13 +50,14 @@ class HaloDBFile { private final FileType fileType; private HaloDBFile(int fileId, File backingFile, DBDirectory dbDirectory, IndexFile indexFile, FileType fileType, - FileChannel channel, HaloDBOptions options) throws IOException { + RandomAccessFile raf, HaloDBOptions options) throws IOException { this.fileId = fileId; this.backingFile = backingFile; this.dbDirectory = dbDirectory; this.indexFile = indexFile; this.fileType = fileType; - this.channel = channel; + this.raf = raf; + this.channel = raf.getChannel(); this.writeOffset = Ints.checkedCast(channel.size()); this.options = options; } @@ -80,20 +82,22 @@ int readFromFile(long position, ByteBuffer destinationBuffer) throws IOException return (int)(currentPosition - position); } - private Record readRecord(int offset) throws HaloDBException, IOException { + private RecordEntry readRecord(int offset) throws HaloDBException, IOException { long tempOffset = offset; // read the header from disk. - ByteBuffer headerBuf = ByteBuffer.allocate(Record.Header.HEADER_SIZE); + ByteBuffer headerBuf = ByteBuffer.allocate(RecordEntry.Header.HEADER_SIZE); int readSize = readFromFile(offset, headerBuf); - if (readSize != Record.Header.HEADER_SIZE) { + if (readSize != RecordEntry.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); + RecordEntry.Header header; + try { + header = RecordEntry.Header.deserialize(headerBuf); + } catch (IllegalArgumentException e) { + throw new HaloDBException("Corrupted header at " + offset + " in file " + fileId, e); } // read key-value from disk. @@ -103,14 +107,11 @@ private Record readRecord(int offset) throws HaloDBException, IOException { 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())); + RecordEntry record = RecordEntry.deserialize(header, recordBuf); return record; } - InMemoryIndexMetaData writeRecord(Record record) throws IOException { + InMemoryIndexMetaData writeRecord(RecordEntry record) throws IOException { writeToChannel(record.serialize()); int recordSize = record.getRecordSize(); @@ -118,14 +119,13 @@ InMemoryIndexMetaData writeRecord(Record record) throws IOException { writeOffset += recordSize; IndexFileEntry indexFileEntry = new IndexFileEntry( - record.getKey(), recordSize, + 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()); + return new InMemoryIndexMetaData(indexFileEntry, fileId); } void rebuildIndexFile() throws IOException { @@ -137,7 +137,7 @@ void rebuildIndexFile() throws IOException { HaloDBFileIterator iterator = new HaloDBFileIterator(); int offset = 0; while (iterator.hasNext()) { - Record record = iterator.next(); + RecordEntry record = iterator.next(); IndexFileEntry indexFileEntry = new IndexFileEntry( record.getKey(), record.getRecordSize(), offset, record.getSequenceNumber(), @@ -162,7 +162,7 @@ HaloDBFile repairFile(DBDirectory dbDirectory) throws IOException { HaloDBFileIterator iterator = new HaloDBFileIterator(); int count = 0; while (iterator.hasNext()) { - Record record = iterator.next(); + RecordEntry record = iterator.next(); // if the header is corrupted iterator will return null. if (record != null && record.verifyChecksum()) { repairFile.writeRecord(record); @@ -191,10 +191,10 @@ private HaloDBFile createRepairFile() throws IOException { repairFile.delete(); } - FileChannel channel = new RandomAccessFile(repairFile, "rw").getChannel(); + RandomAccessFile raf = new RandomAccessFile(repairFile, "rw"); IndexFile indexFile = new IndexFile(fileId, dbDirectory, options); indexFile.createRepairFile(); - return new HaloDBFile(fileId, repairFile, dbDirectory, indexFile, fileType, channel, options); + return new HaloDBFile(fileId, repairFile, dbDirectory, indexFile, fileType, raf, options); } private long writeToChannel(ByteBuffer[] buffers) throws IOException { @@ -252,11 +252,11 @@ int getFileId() { 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(); + RandomAccessFile raf = new RandomAccessFile(filename, "r"); IndexFile indexFile = new IndexFile(fileId, dbDirectory, options); indexFile.open(); - return new HaloDBFile(fileId, filename, dbDirectory, indexFile, fileType, channel, options); + return new HaloDBFile(fileId, filename, dbDirectory, indexFile, fileType, raf, options); } static HaloDBFile create(DBDirectory dbDirectory, int fileId, HaloDBOptions options, FileType fileType) throws IOException { @@ -269,14 +269,14 @@ static HaloDBFile create(DBDirectory dbDirectory, int fileId, HaloDBOptions opti file = toFile.apply(dbDirectory, fileId); } - FileChannel channel = new RandomAccessFile(file, "rw").getChannel(); + RandomAccessFile raf = new RandomAccessFile(file, "rw"); //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); + return new HaloDBFile(fileId, file, dbDirectory, indexFile, fileType, raf, options); } HaloDBFileIterator newIterator() throws IOException { @@ -284,8 +284,8 @@ HaloDBFileIterator newIterator() throws IOException { } void close() throws IOException { - if (channel != null) { - channel.close(); + if (raf != null) { + raf.close(); } if (indexFile != null) { indexFile.close(); @@ -349,8 +349,8 @@ public boolean hasNext() { } @Override - public Record next() { - Record record; + public RecordEntry next() { + RecordEntry record; try { record = readRecord(currentOffset); } catch (IOException | HaloDBException e) { diff --git a/src/main/java/com/oath/halodb/HaloDBInternal.java b/src/main/java/com/oath/halodb/HaloDBInternal.java index a558cf0..9d5b321 100644 --- a/src/main/java/com/oath/halodb/HaloDBInternal.java +++ b/src/main/java/com/oath/halodb/HaloDBInternal.java @@ -5,17 +5,6 @@ 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; @@ -23,6 +12,9 @@ import java.nio.channels.FileChannel; import java.nio.channels.FileLock; import java.nio.channels.OverlappingFileLockException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; import java.nio.file.StandardOpenOption; import java.util.ArrayList; import java.util.Comparator; @@ -32,11 +24,24 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.*; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import java.util.regex.Matcher; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.primitives.Ints; +import com.google.common.util.concurrent.RateLimiter; class HaloDBInternal { @@ -51,7 +56,7 @@ class HaloDBInternal { private volatile Thread tombstoneMergeThread; - private Map readFileMap = new ConcurrentHashMap<>(); + private final Map readFileMap = new ConcurrentHashMap<>(); HaloDBOptions options; @@ -216,16 +221,10 @@ synchronized void close() throws IOException { } 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. + //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); + RecordEntry record = RecordEntry.newEntry(key, value, getNextSequenceNumber()); InMemoryIndexMetaData entry = writeRecordToFile(record); markPreviousVersionAsStale(key); @@ -262,7 +261,7 @@ byte[] get(byte[] key, int attemptNumber) throws IOException, HaloDBException { return get(key, attemptNumber+1); } - // trying to read after HaloDB.close() method called. + // trying to read after HaloDB.close() method called. throw e; } } @@ -425,13 +424,13 @@ void resumeCompaction() { compactionManager.resumeCompaction(); } - private InMemoryIndexMetaData writeRecordToFile(Record record) throws IOException, HaloDBException { + private InMemoryIndexMetaData writeRecordToFile(RecordEntry 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; + int size = record.getKey().length + record.getValue().length + RecordEntry.Header.HEADER_SIZE; if ((currentWriteFile == null || currentWriteFile.getWriteOffset() + size > options.getMaxFileSize()) && !isClosing) { forceRollOverCurrentWriteFile(); @@ -477,7 +476,7 @@ private void markPreviousVersionAsStale(byte[] key) { } private void markPreviousVersionAsStale(byte[] key, InMemoryIndexMetaData recordMetaData) { - int staleRecordSize = Utils.getRecordSize(key.length, recordMetaData.getValueSize()); + int staleRecordSize = RecordEntry.getRecordSize(key.length, recordMetaData.getValueSize()); addFileToCompactionQueueIfThresholdCrossed(recordMetaData.getFileId(), staleRecordSize); } @@ -639,28 +638,24 @@ public Long call() throws IOException { int count = 0, inserted = 0; while (iterator.hasNext()) { IndexFileEntry indexFileEntry = iterator.next(); + InMemoryIndexMetaData metaData = new InMemoryIndexMetaData(indexFileEntry, fileId); 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); + addFileToCompactionQueueIfThresholdCrossed(fileId, indexFileEntry.getRecordSize()); break; } if (inMemoryIndex.replace(key, existing, metaData)) { // update stale data map for the previous version. - addFileToCompactionQueueIfThresholdCrossed(existing.getFileId(), Utils.getRecordSize(key.length, existing.getValueSize())); + addFileToCompactionQueueIfThresholdCrossed( + existing.getFileId(),RecordEntry.getRecordSize(key.length, existing.getValueSize())); inserted++; break; } @@ -707,7 +702,7 @@ public Long call() throws IOException { // update stale data map for the previous version. addFileToCompactionQueueIfThresholdCrossed( - existing.getFileId(), Utils.getRecordSize(key.length, existing.getValueSize())); + existing.getFileId(), RecordEntry.getRecordSize(key.length, existing.getValueSize())); active++; if (options.isCleanUpTombstonesDuringOpen()) { diff --git a/src/main/java/com/oath/halodb/HaloDBIterator.java b/src/main/java/com/oath/halodb/HaloDBIterator.java index 1469507..ea51953 100644 --- a/src/main/java/com/oath/halodb/HaloDBIterator.java +++ b/src/main/java/com/oath/halodb/HaloDBIterator.java @@ -5,22 +5,22 @@ 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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + public class HaloDBIterator implements Iterator { private static final Logger logger = LoggerFactory.getLogger(HaloDBIterator.class); - private Iterator outer; + private final Iterator outer; private Iterator inner; private HaloDBFile currentFile; - private Record next; + private RecordIterated next; private final HaloDBInternal dbInternal; @@ -112,16 +112,14 @@ private boolean readNextRecord() { return false; } - private Record readRecordFromDataFile(IndexFileEntry entry) throws IOException { - InMemoryIndexMetaData meta = Utils.getMetaData(entry, currentFile.getFileId()); - Record record = null; + private RecordIterated readRecordFromDataFile(IndexFileEntry entry) throws IOException { + InMemoryIndexMetaData meta = new InMemoryIndexMetaData(entry, currentFile.getFileId()); 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); + RecordEntry.getValueOffset(entry.getRecordOffset(), entry.getKey().length), + RecordEntry.getValueSize(entry.getRecordSize(), entry.getKey().length)); + return new RecordIterated(entry.getKey(), value, entry.getSequenceNumber()); } - return record; + return null; } } diff --git a/src/main/java/com/oath/halodb/HashEntry.java b/src/main/java/com/oath/halodb/HashEntry.java new file mode 100644 index 0000000..8532344 --- /dev/null +++ b/src/main/java/com/oath/halodb/HashEntry.java @@ -0,0 +1,78 @@ +/* + * Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms. + */ + +package com.oath.halodb; + +/** + * The key size and value size stored in the in-memory index for each key. + * + * The serialization of these values is shared across all entry types. + * Different segment implementations may store the key and value sizes + * in different locations relative to the location information. + */ +abstract class HashEntry { + /* + * key and value size - 5 bytes, 11 bits for key size and 29 bits for value size + */ + static final int ENTRY_SIZES_SIZE = 5; + + private final short keySize; + private final int valueSize; + + HashEntry(int keySize, int valueSize) { + this.keySize = Utils.validateKeySize(keySize); + this.valueSize = Utils.validateValueSize(valueSize); + } + + static short readKeySize(long address) { + return extractKeySize(Uns.getInt(address, 0)); + } + + static short extractKeySize(int firstWord) { + return (short) (firstWord & 0b0111_1111_1111); + } + + static int extractValueSize(int firstWord, byte nextByte) { + return (firstWord >>> 11) | nextByte << 21; + } + + static boolean compareSizes(long address, short keySize, int valueSize) { + int firstWord = Uns.getInt(address, 0); + return keySize == extractKeySize(firstWord) + && valueSize == extractValueSize(firstWord, Uns.getByte(address, 4)); + } + + static void serializeSizes(long address, short keySize, int valueSize) { + Uns.putInt(address, 0, keySize | valueSize << 11); + Uns.putByte(address, 4, (byte) (valueSize >>> 21)); + } + + final short getKeySize() { + return keySize; + } + + final int getValueSize() { + return valueSize; + } + + final boolean compareSizes(long address) { + return compareSizes(address, getKeySize(), getValueSize()); + } + + final void serializeSizes(long address) { + serializeSizes(address, getKeySize(), getValueSize()); + } + + final boolean compare(long sizeAddress, long locationAddress) { + return compareLocation(locationAddress) && compareSizes(sizeAddress); + } + + /** write the location data to memory at the given address **/ + abstract void serializeLocation(long locationAddress); + + /** return true if this entry's location data matches the data at the given address **/ + abstract boolean compareLocation(long locationAddress); +} + + diff --git a/src/main/java/com/oath/halodb/HashEntryLocation.java b/src/main/java/com/oath/halodb/HashEntryLocation.java new file mode 100644 index 0000000..654d0fc --- /dev/null +++ b/src/main/java/com/oath/halodb/HashEntryLocation.java @@ -0,0 +1,48 @@ +/* + * Licensed under the terms of the Apache License 2.0. Please refer to accompanying LICENSE file for terms. + */ + +package com.oath.halodb; + +/** + * Metadata stored in the in-memory index for referencing the location of a record in a file. + */ +interface HashEntryLocation { + + /* + * file id - 4 bytes + * value offset - 4 bytes + * sequence number - 8 bytes + */ + int ENTRY_LOCATION_SIZE = 4 + 4 + 8; + + static void serializeLocation(long address, int fileId, int valueOffset, long sequenceNumber) { + Uns.putInt(address, 0, fileId); + Uns.putInt(address, 4, valueOffset); + Uns.putLong(address, 8, sequenceNumber); + } + + static boolean compareLocation(long address, int fileId, int valueOffset, long sequenceNumber) { + return fileId == Uns.getInt(address, 0) + && valueOffset == Uns.getInt(address, 4) + && sequenceNumber == Uns.getLong(address, 8); + } + + static int readFileId(long address) { + return Uns.getInt(address, 0); + } + + static int readValueOffset(long address) { + return Uns.getInt(address, 4); + } + + static long readSequenceNumber(long address) { + return Uns.getLong(address, 8); + } + + int getFileId(); + + int getValueOffset(); + + long getSequenceNumber(); +} diff --git a/src/main/java/com/oath/halodb/HashEntrySerializer.java b/src/main/java/com/oath/halodb/HashEntrySerializer.java new file mode 100644 index 0000000..c1c926c --- /dev/null +++ b/src/main/java/com/oath/halodb/HashEntrySerializer.java @@ -0,0 +1,49 @@ +/* + * 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; + +/** + * Serialize and deserialize cache entry data. + * + * The key and value size data are stored independently from + * the location payload. Different Segment implementations may + * store each of these at different relative places in their + * hash slot. + */ +abstract class HashEntrySerializer { + + /** read the key size from the given address **/ + final short readKeySize(long sizeAddress) { + return HashEntry.readKeySize(sizeAddress); + } + + /** the serialized size of the key and value length **/ + final int sizesSize() { + return HashEntry.ENTRY_SIZES_SIZE; + } + + /** the total size of the entry, including sizes and location data **/ + final int entrySize() { + return sizesSize() + locationSize(); + } + + /** read the entry from memory, from the provided sizeAddress and locationAddress **/ + abstract E deserialize(long sizeAddress, long locationAddress); + + /** the size of the location data **/ + abstract int locationSize(); + + final void validateSize(E entry) { + if (!validSize(entry)) { + throw new IllegalArgumentException("value size incompatible with fixed value size " + entrySize()); + } + } + + /** return true if the entry serializes to entrySize bytes **/ + abstract boolean validSize(E entry); +} diff --git a/src/main/java/com/oath/halodb/HashTableUtil.java b/src/main/java/com/oath/halodb/HashTableUtil.java index 89d7188..ce5b943 100644 --- a/src/main/java/com/oath/halodb/HashTableUtil.java +++ b/src/main/java/com/oath/halodb/HashTableUtil.java @@ -18,17 +18,15 @@ 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 int bitNum(int val) { + return 32 - Integer.numberOfLeadingZeros(val); } - static long roundUpToPowerOf2(long number, long max) { - return number >= max - ? max - : (number > 1) ? Long.highestOneBit((number - 1) << 1) : 1; + static int roundUpToPowerOf2(int posNum, int maxPower) { + int max = 1 << maxPower; + if (posNum >= max) { + return max; + } + return 1 << bitNum(posNum - 1); } } diff --git a/src/main/java/com/oath/halodb/HashTableValueSerializer.java b/src/main/java/com/oath/halodb/HashTableValueSerializer.java deleted file mode 100644 index b962d44..0000000 --- a/src/main/java/com/oath/halodb/HashTableValueSerializer.java +++ /dev/null @@ -1,23 +0,0 @@ -/* - * 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); -} - diff --git a/src/main/java/com/oath/halodb/InMemoryIndex.java b/src/main/java/com/oath/halodb/InMemoryIndex.java index b48ba4c..b60fe0e 100644 --- a/src/main/java/com/oath/halodb/InMemoryIndex.java +++ b/src/main/java/com/oath/halodb/InMemoryIndex.java @@ -5,12 +5,12 @@ package com.oath.halodb; -import com.google.common.primitives.Ints; +import java.io.IOException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; +import com.google.common.primitives.Ints; /** * Hash table stored in native memory, outside Java heap. @@ -28,11 +28,9 @@ class InMemoryIndex { maxSizeOfEachSegment = Ints.checkedCast(Utils.roundUpToPowerOf2(numberOfKeys / noOfSegments)); long start = System.currentTimeMillis(); OffHeapHashTableBuilder builder = - OffHeapHashTableBuilder.newBuilder() - .valueSerializer(new InMemoryIndexMetaDataSerializer()) + OffHeapHashTableBuilder.newBuilder(new InMemoryIndexMetaDataSerializer()) .segmentCount(noOfSegments) .hashTableSize(maxSizeOfEachSegment) - .fixedValueSize(InMemoryIndexMetaData.SERIALIZED_SIZE) .loadFactor(1); if (useMemoryPool) { diff --git a/src/main/java/com/oath/halodb/InMemoryIndexMetaData.java b/src/main/java/com/oath/halodb/InMemoryIndexMetaData.java index 53c6375..dfbf5fd 100644 --- a/src/main/java/com/oath/halodb/InMemoryIndexMetaData.java +++ b/src/main/java/com/oath/halodb/InMemoryIndexMetaData.java @@ -6,57 +6,76 @@ package com.oath.halodb; -import java.nio.ByteBuffer; - /** * Metadata stored in the in-memory index for each key. */ -class InMemoryIndexMetaData { +class InMemoryIndexMetaData extends HashEntry implements HashEntryLocation { 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; + /* + * From HashEntry: + * key and value size - 5 bytes, 11 bits for key size and 29 bits for value size + * + * Additionally in this class, value meta: + * file id - 4 bytes + * value offset - 4 bytes + * sequence number - 8 bytes + */ + static final int VALUE_META_SIZE = 4 + 4 + 8; - InMemoryIndexMetaData(int fileId, int valueOffset, int valueSize, long sequenceNumber) { + InMemoryIndexMetaData(int fileId, int valueOffset, int valueSize, long sequenceNumber, int keySize) { + super(keySize, valueSize); 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(); + InMemoryIndexMetaData(IndexFileEntry entry, int fileId) { + this(fileId, + RecordEntry.getValueOffset(entry.getRecordOffset(), entry.getKey().length), + RecordEntry.getValueSize(entry.getRecordSize(), entry.getKey().length), + entry.getSequenceNumber(), + entry.getKey().length); } - 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); + InMemoryIndexMetaData(RecordEntry.Header header, int fileId, int offset) { + this(fileId, + RecordEntry.getValueOffset(offset, header.getKeySize()), + header.getValueSize(), + header.getSequenceNumber(), + header.getKeySize()); } - int getFileId() { + @Override + public int getFileId() { return fileId; } - int getValueOffset() { + @Override + public int getValueOffset() { return valueOffset; } - int getValueSize() { - return valueSize; + @Override + public long getSequenceNumber() { + return sequenceNumber; } - long getSequenceNumber() { - return sequenceNumber; + @Override + final boolean compareLocation(long address) { + return HashEntryLocation.compareLocation(address, getFileId(), getValueOffset(), getSequenceNumber()); + } + + @Override + final void serializeLocation(long address) { + HashEntryLocation.serializeLocation(address, getFileId(), getValueOffset(), getSequenceNumber()); + } + + InMemoryIndexMetaData relocated(int newFileId, int newWriteFileOffset) { + int newOffset = RecordEntry.getValueOffset(newWriteFileOffset, getKeySize()); + return new InMemoryIndexMetaData(newFileId, newOffset, getValueSize(), sequenceNumber, getKeySize()); } } diff --git a/src/main/java/com/oath/halodb/InMemoryIndexMetaDataSerializer.java b/src/main/java/com/oath/halodb/InMemoryIndexMetaDataSerializer.java index a11e3ee..898e28a 100644 --- a/src/main/java/com/oath/halodb/InMemoryIndexMetaDataSerializer.java +++ b/src/main/java/com/oath/halodb/InMemoryIndexMetaDataSerializer.java @@ -5,20 +5,27 @@ package com.oath.halodb; -import java.nio.ByteBuffer; +class InMemoryIndexMetaDataSerializer extends HashEntrySerializer { -class InMemoryIndexMetaDataSerializer implements HashTableValueSerializer { - - public void serialize(InMemoryIndexMetaData recordMetaData, ByteBuffer byteBuffer) { - recordMetaData.serialize(byteBuffer); - byteBuffer.flip(); + @Override + final InMemoryIndexMetaData deserialize(long sizeAddress, long locationAddress) { + int firstWord = Uns.getInt(sizeAddress, 0); + byte nextByte = Uns.getByte(sizeAddress, 4); + short keySize = HashEntry.extractKeySize(firstWord); + int valueSize = HashEntry.extractValueSize(firstWord, nextByte); + int fileId = HashEntryLocation.readFileId(locationAddress); + int valueOffset = HashEntryLocation.readValueOffset(locationAddress); + long sequenceNumber = HashEntryLocation.readSequenceNumber(locationAddress); + return new InMemoryIndexMetaData(fileId, valueOffset, valueSize, sequenceNumber, keySize); } - public InMemoryIndexMetaData deserialize(ByteBuffer byteBuffer) { - return InMemoryIndexMetaData.deserialize(byteBuffer); + @Override + final int locationSize() { + return HashEntryLocation.ENTRY_LOCATION_SIZE; } - public int serializedSize(InMemoryIndexMetaData recordMetaData) { - return InMemoryIndexMetaData.SERIALIZED_SIZE; + @Override + boolean validSize(InMemoryIndexMetaData entry) { + return true; } } diff --git a/src/main/java/com/oath/halodb/IndexFile.java b/src/main/java/com/oath/halodb/IndexFile.java index ee6e49b..fa76a96 100644 --- a/src/main/java/com/oath/halodb/IndexFile.java +++ b/src/main/java/com/oath/halodb/IndexFile.java @@ -5,19 +5,19 @@ package com.oath.halodb; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; +import java.io.FileNotFoundException; 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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + class IndexFile { private static final Logger logger = LoggerFactory.getLogger(IndexFile.class); @@ -25,6 +25,7 @@ class IndexFile { private final DBDirectory dbDirectory; private File backingFile; + private RandomAccessFile raf; private FileChannel channel; private final HaloDBOptions options; @@ -45,7 +46,7 @@ void create() throws IOException { if (!backingFile.createNewFile()) { throw new IOException("Index file with id " + fileId + " already exists"); } - channel = new RandomAccessFile(backingFile, "rw").getChannel(); + openFile(); } void createRepairFile() throws IOException { @@ -54,25 +55,31 @@ void createRepairFile() throws IOException { 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(); + openFile(); } void open() throws IOException { backingFile = getIndexFile(); - channel = new RandomAccessFile(backingFile, "rw").getChannel(); + openFile(); + } + + private void openFile() throws FileNotFoundException { + raf = new RandomAccessFile(backingFile, "rw"); + channel = raf.getChannel(); } void close() throws IOException { - if (channel != null) { - channel.close(); + if (raf != null) { + // this closes the associated channel as well + raf.close(); } } void delete() throws IOException { - if (channel != null && channel.isOpen()) - channel.close(); - - getIndexFile().delete(); + close(); + if (backingFile != null) { + backingFile.delete(); + } } void write(IndexFileEntry entry) throws IOException { diff --git a/src/main/java/com/oath/halodb/IndexFileEntry.java b/src/main/java/com/oath/halodb/IndexFileEntry.java index c47a1bc..a3ab9b8 100644 --- a/src/main/java/com/oath/halodb/IndexFileEntry.java +++ b/src/main/java/com/oath/halodb/IndexFileEntry.java @@ -14,12 +14,11 @@ class IndexFileEntry { /** - * checksum - 4 bytes. - * version - 1 byte. - * Key size - 1 bytes. - * record size - 4 bytes. - * record offset - 4 bytes. - * sequence number - 8 bytes + * checksum - 4 bytes. + * version + key size - 2 bytes. 5 bits for version, 11 for keySize + * 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; @@ -31,47 +30,47 @@ class IndexFileEntry { 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; + private final byte version; - IndexFileEntry(byte[] key, int recordSize, int recordOffset, long sequenceNumber, int version, long checkSum) { + IndexFileEntry(byte[] key, int recordSize, int recordOffset, long sequenceNumber, byte 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; + private ByteBuffer serializeHeader() { + ByteBuffer header = ByteBuffer.allocate(INDEX_FILE_HEADER_SIZE); + header.put(VERSION_OFFSET, Utils.versionByte(version, key.length)); + header.put(KEY_SIZE_OFFSET, Utils.keySizeByte(key.length)); + header.putInt(RECORD_SIZE_OFFSET, recordSize); + header.putInt(RECORD_OFFSET, recordOffset); + header.putLong(SEQUENCE_NUMBER_OFFSET, sequenceNumber); + return header; } 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) }; + ByteBuffer header = serializeHeader(); + long crc32 = computeCheckSum(header.array()); + header.putInt(CHECKSUM_OFFSET, Utils.toSignedIntFromLong(crc32)); + return new ByteBuffer[] { header, ByteBuffer.wrap(key) }; } static IndexFileEntry deserialize(ByteBuffer buffer) { long crc32 = Utils.toUnsignedIntFromInt(buffer.getInt()); - int version = Utils.toUnsignedByte(buffer.get()); - byte keySize = buffer.get(); + byte vbyte = buffer.get(); + byte keySizeByte = buffer.get(); int recordSize = buffer.getInt(); int offset = buffer.getInt(); + byte version = Utils.version(vbyte); + short keySize = Utils.keySize(vbyte, keySizeByte); long sequenceNumber = buffer.getLong(); byte[] key = new byte[keySize]; @@ -84,15 +83,17 @@ 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(); + byte vbyte = buffer.get(); + byte keySizeByte = buffer.get(); int recordSize = buffer.getInt(); int offset = buffer.getInt(); + byte version = Utils.version(vbyte); + short keySize = Utils.keySize(vbyte, keySizeByte); long sequenceNumber = buffer.getLong(); - if (sequenceNumber < 0 || keySize <= 0 - || version < 0 || version > 255 + + if (sequenceNumber < 0 || keySize < 0 + || version < 0 || version > 31 || recordSize <= 0 || offset < 0 || buffer.remaining() < keySize) { return null; @@ -117,12 +118,7 @@ private long computeCheckSum(byte[] header) { } 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); + ByteBuffer header = serializeHeader(); return computeCheckSum(header.array()); } @@ -142,7 +138,7 @@ long getSequenceNumber() { return sequenceNumber; } - int getVersion() { + byte getVersion() { return version; } diff --git a/src/main/java/com/oath/halodb/KeyBuffer.java b/src/main/java/com/oath/halodb/KeyBuffer.java index 8b6fd8f..6e1cf6a 100644 --- a/src/main/java/com/oath/halodb/KeyBuffer.java +++ b/src/main/java/com/oath/halodb/KeyBuffer.java @@ -24,10 +24,10 @@ long hash() { KeyBuffer finish(Hasher hasher) { hash = hasher.hash(buffer); - return this; } + @Override public boolean equals(Object o) { if (this == o) { return true; @@ -45,6 +45,7 @@ public int size() { return buffer.length; } + @Override public int hashCode() { return (int) hash; } @@ -70,39 +71,4 @@ public String toString() { } 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; - } } diff --git a/src/main/java/com/oath/halodb/MemoryPoolAddress.java b/src/main/java/com/oath/halodb/MemoryPoolAddress.java index eee79e5..c24412f 100644 --- a/src/main/java/com/oath/halodb/MemoryPoolAddress.java +++ b/src/main/java/com/oath/halodb/MemoryPoolAddress.java @@ -6,33 +6,46 @@ 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. + * Represents the address of an entry in the memory pool. + * + * 1 byte -- chunkIndex as an int between 0 and 255, valid chunks are 1 to 255, 0 indicates an empty address + * 3 bytes -- slot as an int between 0 and 2^24-1 (16.77 million). + * + * With slots using 8 byte 'fixedKeyLength', each slot is 33 bytes, and so each chunk in the memory pool + * could hold over 550MB of key data and metadata. There can be 255 slots, so each segment can fit over + * 141GB of data in RAM, and there is typically at least 16 segments. */ -class MemoryPoolAddress { +interface MemoryPoolAddress { + int ADDRESS_SIZE = 4; + int MAX_NUMBER_OF_SLOTS = (1 << 24) - 1; - final byte chunkIndex; - final int chunkOffset; + int empty = 0; - MemoryPoolAddress(byte chunkIndex, int chunkOffset) { - this.chunkIndex = chunkIndex; - this.chunkOffset = chunkOffset; - } - - @Override - public boolean equals(Object o) { - if (o == this) { - return true; + static int encode(int chunkIndex, int slot) { + if ((chunkIndex >>> 8) != 0) { + throw new IllegalArgumentException("Invalid chunk index, must be within [0,255], but was: " + chunkIndex); } - if (!(o instanceof MemoryPoolAddress)) { - return false; + if ((slot & 0xFF00_0000) != 0) { + throw new IllegalArgumentException("Invalid memory pool slot, must be within [0,2^24)" + slot); } - MemoryPoolAddress m = (MemoryPoolAddress) o; - return m.chunkIndex == chunkIndex && m.chunkOffset == chunkOffset; + return chunkIndex << 24 | slot & 0x00FF_FFFF; + } + + /** Always between 0 and (2^24 -1) **/ + static int slot(int memoryPoolAddress) { + return memoryPoolAddress & 0x00FF_FFFF; + } + + /** Always between 0 and 255 **/ + static int chunkIndex(int memoryPoolAddress) { + return memoryPoolAddress >>> 24; + } + + static boolean isEmpty(int memoryPoolAddress) { + return memoryPoolAddress == 0; } - @Override - public int hashCode() { - return 31 * ((31 * chunkIndex) + chunkOffset); + static boolean nonEmpty(int memoryPoolAddress) { + return memoryPoolAddress != 0; } } diff --git a/src/main/java/com/oath/halodb/MemoryPoolChunk.java b/src/main/java/com/oath/halodb/MemoryPoolChunk.java index c510142..992a16e 100644 --- a/src/main/java/com/oath/halodb/MemoryPoolChunk.java +++ b/src/main/java/com/oath/halodb/MemoryPoolChunk.java @@ -5,161 +5,236 @@ 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. + * Memory pool is divided into chunks of configurable sized slots. This represents such a chunk. + * + * All slots are members of linked lists, where the 'next' pointer is the first + * element in the slot. + * + * Slots come in two varieties. If the slot is storing an entry that has a key + * that is less than or equal to the fixedKeyLength, then the key and data all + * fit in one slot. In this case, the slot is as follows: + * + * 4 bytes -- MemoryPoolAddress pointer (next) + * 5 bytes -- HashEntry sizes (key/value length) + * fixedKeyLength bytes -- key data + * 16 bytes -- HashEntry location data (fileId, fileOffset, sequenceId) + * + * If the key is larger than fixedKeyLength bytes, then the data is stored in multiple + * slots in the list, chained together. The remainder of the key 'overflows' into + * additional slots structured as follows: + * + * 4 bytes -- MemoryPoolAddress pointer (next) + * remaining slot bytes -- key fragment + * + * The number of slots that a key of size K requires is + * + * 1 + (K - fixedKeyLength)/(21 + fixedKeyLength) + * + * For example, if fixedKeyLength is 8 bytes, a 60 byte key would require 3 slots: + * 1 + (60 - 8)/(21 + 8) = 1 + (52/29) = 3 + * + * If the fixedKeyLength was 20, a 60 byte key would require 2 slots: + * 1 + (60 - 20)/(21 + 20) = 1 + (40 / 41) = 2 + * */ -class MemoryPoolChunk { +class MemoryPoolChunk { + + private static final int sizesOffset = MemoryPoolAddress.ADDRESS_SIZE; + private final int chunkId; 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) { + private final int fixedKeyOffset; + private final int locationOffset; + private final int slotSize; + private final int slots; + private final HashEntrySerializer serializer; + private int writeSlot = 0; + private boolean destroyed = false; + + private MemoryPoolChunk(long address, int chunkId, int slots, int fixedKeyLength, HashEntrySerializer serializer) { this.address = address; - this.chunkSize = chunkSize; + this.chunkId = chunkId; + this.slots = slots; this.fixedKeyLength = fixedKeyLength; - this.fixedValueLength = fixedValueLength; - this.fixedSlotSize = HEADER_SIZE + fixedKeyLength + fixedValueLength; + this.fixedKeyOffset = sizesOffset + serializer.sizesSize(); + this.locationOffset = fixedKeyOffset + fixedKeyLength; + this.slotSize = MemoryPoolChunk.slotSize(fixedKeyLength, serializer); + this.serializer = serializer; } - 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); + static MemoryPoolChunk create(int id, int chunkSize, int fixedKeyLength, HashEntrySerializer serializer) { + int fixedSlotSize = MemoryPoolChunk.slotSize(fixedKeyLength, serializer); + int slots = Math.min((chunkSize / fixedSlotSize), MemoryPoolAddress.MAX_NUMBER_OF_SLOTS); + if (slots < 1) { + throw new IllegalArgumentException("fixedSlotSize " + fixedSlotSize + " must not be larger than chunkSize " + chunkSize); } - long address = Uns.allocate(chunkSize, true); - return new MemoryPoolChunk(address, chunkSize, fixedKeyLength, fixedValueLength); + long address = Uns.allocate(slots * fixedSlotSize); + return new MemoryPoolChunk<>(address, id, slots, fixedKeyLength, serializer); } 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); + if (!destroyed) { + Uns.free(address); + destroyed = true; + } } - /** - * 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; + @Override + protected void finalize() throws Throwable { + destroy(); + super.finalize(); } - /** - * 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); + Slot slotFor(int slot) { + return new Slot(slot); } - 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) - ); + Slot allocateSlot() { + if (isFull()) { + throw new IllegalStateException("can not allocate a slot when already full"); } - - Uns.copyMemory(value, 0, address, slotOffset + ENTRY_OFF_DATA + fixedKeyLength, value.length); + Slot slot = slotFor(writeSlot); + writeSlot++; + return slot; } int getWriteOffset() { - return writeOffset; + return slotToOffset(writeSlot); } - int remaining() { - return chunkSize - writeOffset; + boolean isFull() { + return writeSlot >= slots; } - ByteBuffer readOnlyValueByteBuffer(int offset) { - return Uns.directBufferFor(address, offset + ENTRY_OFF_DATA + fixedKeyLength, fixedValueLength, true); + int remainingSlots() { + return slots - writeSlot; } - ByteBuffer readOnlyKeyByteBuffer(int offset) { - return Uns.directBufferFor(address, offset + ENTRY_OFF_DATA, getKeyLength(offset), true); + private int slotToOffset(int slot) { + if (slot > slots) { + throw new IllegalArgumentException("Invalid request. slot - " + slot + " total slots - " + slots); + } + return slot * slotSize; } - long computeHash(int slotOffset, Hasher hasher) { - return hasher.hash(address, slotOffset + ENTRY_OFF_DATA, getKeyLength(slotOffset)); + public static int slotSize(int fixedKeySize, HashEntrySerializer serializer) { + return sizesOffset + fixedKeySize + serializer.entrySize(); } + /** Represents a valid Slot within a MemoryPoolChunk **/ + class Slot { + private final int slot; + private final int offset; + private Slot(int slot) { + this.slot = slot; + this.offset = slotToOffset(slot); + } - boolean compareKey(int slotOffset, byte[] key) { - if (key.length > fixedKeyLength || slotOffset + fixedSlotSize > chunkSize) { - throw new IllegalArgumentException("Invalid request. slotOffset - " + slotOffset + " key.length - " + key.length); + short getKeyLength() { + return serializer.readKeySize(sizeAddress()); } - return getKeyLength(slotOffset) == key.length && compare(slotOffset + ENTRY_OFF_DATA, key); - } + int toAddress() { + return MemoryPoolAddress.encode(chunkId, slot); + } - boolean compareValue(int slotOffset, byte[] value) { - if (value.length != fixedValueLength || slotOffset + fixedSlotSize > chunkSize) { - throw new IllegalArgumentException("Invalid request. slotOffset - " + slotOffset + " value.length - " + value.length); + int getNextAddress() { + return Uns.getInt(address, offset); } - return compare(slotOffset + ENTRY_OFF_DATA + fixedKeyLength, value); - } + void setNextAddress(int nextAddress) { + Uns.putInt(address, offset, nextAddress); + } - 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; + void fillSlot(byte[] key, E entry, int nextAddress) { + // pointer to next slot + setNextAddress(nextAddress); + // key and value sizes + entry.serializeSizes(sizeAddress()); + // key data, in fixed slot + setKey(key, Math.min(key.length, fixedKeyLength)); + // entry metadata + entry.serializeLocation(locationAddress()); + } + + void fillOverflowSlot(byte[] key, int keyoffset, int len, int nextAddress) { + //poiner to next slot + setNextAddress(nextAddress); + // set key data + setExtendedKey(key, keyoffset, len); + } + + void setEntry(E entry) { + entry.serializeSizes(sizeAddress()); + entry.serializeLocation(locationAddress()); + } + + E readEntry() { + return serializer.deserialize(sizeAddress(), locationAddress()); + } + + long computeFixedKeyHash(Hasher hasher, int keySize) { + return hasher.hash(keyAddress(), 0, keySize); + } + + void copyEntireFixedKey(long destinationAddress) { + Uns.copyMemory(keyAddress(), 0, destinationAddress, 0, fixedKeyLength); + } + + int copyExtendedKey(long destinationAddress, int destinationOffset, int len) { + int copied = Math.min(len, slotSize - sizesOffset); + Uns.copyMemory(extendedKeyAddress(), 0, destinationAddress, destinationOffset, copied); + return copied; + } + + boolean compareFixedKey(byte[] key, int len) { + if (len > fixedKeyLength) { + throw new IllegalArgumentException("Invalid request. key fragment larger than fixedKeyLength: " + len); } + return Uns.compare(keyAddress(), key, 0, len); } - for (; length - p >= 4; p += 4) { - if (Uns.getInt(address, offset + p) != Uns.getIntFromByteArray(array, p)) { - return false; + + boolean compareExtendedKey(byte[] key, int keyoffset, int len) { + if (len > fixedKeyLength + serializer.entrySize()) { + throw new IllegalArgumentException("Invalid request. key fragment larger than slot capacity: " + len); } + return Uns.compare(extendedKeyAddress(), key, keyoffset, len); } - for (; length - p >= 2; p += 2) { - if (Uns.getShort(address, offset + p) != Uns.getShortFromByteArray(array, p)) { - return false; + + boolean compareEntry(E entry) { + return entry.compare(sizeAddress(), locationAddress()); + } + + private void setKey(byte[] key, int len) { + if (len > fixedKeyLength) { + throw new IllegalArgumentException("Invalid key write beyond fixedKeyLength, length: " + len); } + Uns.copyMemory(key, 0, keyAddress(), 0, len); } - for (; length - p >= 1; p += 1) { - if (Uns.getByte(address, offset + p) != array[p]) { - return false; + + private void setExtendedKey(byte[] key, int keyoffset, int len) { + if (len > slotSize - sizesOffset) { + throw new IllegalArgumentException("Invalid key write beyond slot with extended key, length: " + len); } + Uns.copyMemory(key, keyoffset, extendedKeyAddress(), 0, len); } - return true; - } + private long sizeAddress() { + return address + offset + sizesOffset; + } - private byte getKeyLength(int slotOffset) { - return Uns.getByte(address, slotOffset + ENTRY_OFF_KEY_LENGTH); + private long locationAddress() { + return address + offset + locationOffset; + } + + private long keyAddress() { + return address + offset + fixedKeyOffset; + } + + private long extendedKeyAddress() { + return sizeAddress(); + } } } diff --git a/src/main/java/com/oath/halodb/MemoryPoolHashEntries.java b/src/main/java/com/oath/halodb/MemoryPoolHashEntries.java deleted file mode 100644 index 6dd6869..0000000 --- a/src/main/java/com/oath/halodb/MemoryPoolHashEntries.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * 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; - -} diff --git a/src/main/java/com/oath/halodb/NonMemoryPoolHashEntries.java b/src/main/java/com/oath/halodb/NonMemoryPoolHashEntries.java index 11d065b..e62b081 100644 --- a/src/main/java/com/oath/halodb/NonMemoryPoolHashEntries.java +++ b/src/main/java/com/oath/halodb/NonMemoryPoolHashEntries.java @@ -15,15 +15,11 @@ 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 final long ENTRY_OFF_DATA = 8; - static void init(int keyLen, long hashEntryAdr) { + static void init(long hashEntryAdr) { setNext(hashEntryAdr, 0L); - Uns.putByte(hashEntryAdr, ENTRY_OFF_KEY_LENGTH, (byte) keyLen); } static long getNext(long hashEntryAdr) { @@ -38,8 +34,4 @@ static void setNext(long hashEntryAdr, long nextAdr) { Uns.putLong(hashEntryAdr, ENTRY_OFF_NEXT, nextAdr); } } - - static int getKeyLen(long hashEntryAdr) { - return Uns.getByte(hashEntryAdr, ENTRY_OFF_KEY_LENGTH); - } } diff --git a/src/main/java/com/oath/halodb/OffHeapHashTableBuilder.java b/src/main/java/com/oath/halodb/OffHeapHashTableBuilder.java index ee84ae3..0e50d74 100644 --- a/src/main/java/com/oath/halodb/OffHeapHashTableBuilder.java +++ b/src/main/java/com/oath/halodb/OffHeapHashTableBuilder.java @@ -7,21 +7,21 @@ package com.oath.halodb; -class OffHeapHashTableBuilder { +class OffHeapHashTableBuilder { private int segmentCount; private int hashTableSize = 8192; private int memoryPoolChunkSize = 2 * 1024 * 1024; - private HashTableValueSerializer valueSerializer; + private final HashEntrySerializer serializer; 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() { + private OffHeapHashTableBuilder(HashEntrySerializer serializer) { + this.serializer = serializer; int cpus = Runtime.getRuntime().availableProcessors(); segmentCount = roundUpToPowerOf2(cpus * 2, 1 << 30); @@ -35,21 +35,17 @@ static int roundUpToPowerOf2(int number, int max) { : (number > 1) ? Integer.highestOneBit((number - 1) << 1) : 1; } - static OffHeapHashTableBuilder newBuilder() { - return new OffHeapHashTableBuilder<>(); + static OffHeapHashTableBuilder newBuilder(HashEntrySerializer serializer) { + return new OffHeapHashTableBuilder<>(serializer); } - public OffHeapHashTable build() { - if (fixedValueSize == -1) { - throw new IllegalArgumentException("Need to set fixedValueSize"); - } - + public OffHeapHashTable build() { //TODO: write a test. if (useMemoryPool && fixedKeySize == -1) { throw new IllegalArgumentException("Need to set fixedKeySize when using memory pool"); } - if (valueSerializer == null) { + if (serializer == null) { throw new IllegalArgumentException("Value serializer must be set."); } @@ -60,7 +56,7 @@ public int getHashTableSize() { return hashTableSize; } - public OffHeapHashTableBuilder hashTableSize(int hashTableSize) { + public OffHeapHashTableBuilder hashTableSize(int hashTableSize) { if (hashTableSize < -1) { throw new IllegalArgumentException("hashTableSize:" + hashTableSize); } @@ -72,7 +68,7 @@ public int getMemoryPoolChunkSize() { return memoryPoolChunkSize; } - public OffHeapHashTableBuilder memoryPoolChunkSize(int chunkSize) { + public OffHeapHashTableBuilder memoryPoolChunkSize(int chunkSize) { if (chunkSize < -1) { throw new IllegalArgumentException("memoryPoolChunkSize:" + chunkSize); } @@ -80,20 +76,15 @@ public OffHeapHashTableBuilder memoryPoolChunkSize(int chunkSize) { return this; } - public HashTableValueSerializer getValueSerializer() { - return valueSerializer; - } - - public OffHeapHashTableBuilder valueSerializer(HashTableValueSerializer valueSerializer) { - this.valueSerializer = valueSerializer; - return this; + public HashEntrySerializer getEntrySerializer() { + return serializer; } public int getSegmentCount() { return segmentCount; } - public OffHeapHashTableBuilder segmentCount(int segmentCount) { + public OffHeapHashTableBuilder segmentCount(int segmentCount) { if (segmentCount < -1) { throw new IllegalArgumentException("segmentCount:" + segmentCount); } @@ -105,7 +96,7 @@ public float getLoadFactor() { return loadFactor; } - public OffHeapHashTableBuilder loadFactor(float loadFactor) { + public OffHeapHashTableBuilder loadFactor(float loadFactor) { if (loadFactor <= 0f) { throw new IllegalArgumentException("loadFactor:" + loadFactor); } @@ -117,7 +108,7 @@ public int getFixedKeySize() { return fixedKeySize; } - public OffHeapHashTableBuilder fixedKeySize(int fixedKeySize) { + public OffHeapHashTableBuilder fixedKeySize(int fixedKeySize) { if (fixedKeySize <= 0) { throw new IllegalArgumentException("fixedValueSize:" + fixedKeySize); } @@ -125,18 +116,6 @@ public OffHeapHashTableBuilder fixedKeySize(int 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; } @@ -145,7 +124,7 @@ public Hasher getHasher() { return hasher; } - public OffHeapHashTableBuilder hashMode(HashAlgorithm hashMode) { + public OffHeapHashTableBuilder hashMode(HashAlgorithm hashMode) { if (hashMode == null) { throw new NullPointerException("hashMode"); } @@ -158,7 +137,7 @@ public boolean isUnlocked() { return unlocked; } - public OffHeapHashTableBuilder unlocked(boolean unlocked) { + public OffHeapHashTableBuilder unlocked(boolean unlocked) { this.unlocked = unlocked; return this; } @@ -167,7 +146,7 @@ public boolean isUseMemoryPool() { return useMemoryPool; } - public OffHeapHashTableBuilder useMemoryPool(boolean useMemoryPool) { + public OffHeapHashTableBuilder useMemoryPool(boolean useMemoryPool) { this.useMemoryPool = useMemoryPool; return this; } diff --git a/src/main/java/com/oath/halodb/OffHeapHashTableImpl.java b/src/main/java/com/oath/halodb/OffHeapHashTableImpl.java index cab946a..c63a78b 100644 --- a/src/main/java/com/oath/halodb/OffHeapHashTableImpl.java +++ b/src/main/java/com/oath/halodb/OffHeapHashTableImpl.java @@ -7,26 +7,20 @@ package com.oath.halodb; -import com.google.common.primitives.Ints; -import com.oath.halodb.histo.EstimatedHistogram; +import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; +import com.oath.halodb.histo.EstimatedHistogram; -final class OffHeapHashTableImpl implements OffHeapHashTable { +final class OffHeapHashTableImpl implements OffHeapHashTable { private static final Logger logger = LoggerFactory.getLogger(OffHeapHashTableImpl.class); - private final HashTableValueSerializer valueSerializer; + private final HashEntrySerializer serializer; - private final int fixedValueLength; - - private final List> segments; + private final Segment[] segments; private final long segmentMask; private final int segmentShift; @@ -34,27 +28,26 @@ final class OffHeapHashTableImpl implements OffHeapHashTable { private volatile long putFailCount; - private boolean closed; + private boolean closed = false; private final Hasher hasher; - OffHeapHashTableImpl(OffHeapHashTableBuilder builder) { + 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); + segmentCount = HashTableUtil.roundUpToPowerOf2(builder.getSegmentCount(), 30); + segments = new Segment[segmentCount]; for (int i = 0; i < segmentCount; i++) { try { - segments.add(allocateSegment(builder)); + segments[i] = (allocateSegment(builder)); } catch (RuntimeException e) { for (; i >= 0; i--) { - if (segments.get(i) != null) { - segments.get(i).release(); + if (segments[i] != null) { + segments[i].release(); } } throw e; @@ -66,22 +59,23 @@ final class OffHeapHashTableImpl implements OffHeapHashTable { this.segmentShift = 64 - bitNum; this.segmentMask = ((long) segmentCount - 1) << segmentShift; - this.valueSerializer = builder.getValueSerializer(); - if (valueSerializer == null) { - throw new NullPointerException("valueSerializer == null"); + this.serializer = builder.getEntrySerializer(); + if (serializer == null) { + throw new NullPointerException("serializer == null"); } logger.debug("off-heap index with {} segments created.", segmentCount); } - private Segment allocateSegment(OffHeapHashTableBuilder builder) { + private Segment allocateSegment(OffHeapHashTableBuilder builder) { if (builder.isUseMemoryPool()) { return new SegmentWithMemoryPool<>(builder); } return new SegmentNonMemoryPool<>(builder); } - public V get(byte[] key) { + @Override + public E get(byte[] key) { if (key == null) { throw new NullPointerException(); } @@ -90,6 +84,7 @@ public V get(byte[] key) { return segment(keySource.hash()).getEntry(keySource); } + @Override public boolean containsKey(byte[] key) { if (key == null) { throw new NullPointerException(); @@ -99,48 +94,36 @@ public boolean containsKey(byte[] key) { return segment(keySource.hash()).containsEntry(keySource); } - public boolean put(byte[] k, V v) { + @Override + public boolean put(byte[] k, E v) { return putInternal(k, v, false, null); } - public boolean addOrReplace(byte[] key, V old, V value) { - return putInternal(key, value, false, old); + @Override + public boolean addOrReplace(byte[] key, E old, E entry) { + return putInternal(key, entry, false, old); } - public boolean putIfAbsent(byte[] k, V v) { + @Override + public boolean putIfAbsent(byte[] k, E v) { return putInternal(k, v, true, null); } - private boolean putInternal(byte[] key, V value, boolean ifAbsent, V old) { - if (key == null || value == null) { + private boolean putInternal(byte[] key, E entry, boolean ifAbsent, E old) { + if (key == null || entry == 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); + Utils.validateKeySize(key.length); + serializer.validateSize(entry); + if (old != null) { + serializer.validateSize(entry); } - - 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; + KeyBuffer k = new KeyBuffer(key); + k.finish(hasher); + return segment(k.hash()).putEntry(k, entry, ifAbsent, old); } + @Override public boolean remove(byte[] k) { if (k == null) { throw new NullPointerException(); @@ -150,9 +133,9 @@ public boolean remove(byte[] k) { return segment(keySource.hash()).removeEntry(keySource); } - private Segment segment(long hash) { + private Segment segment(long hash) { int seg = (int) ((hash & segmentMask) >>> segmentShift); - return segments.get(seg); + return segments[seg]; } private KeyBuffer keySource(byte[] key) { @@ -164,8 +147,9 @@ private KeyBuffer keySource(byte[] key) { // maintenance // + @Override public void clear() { - for (Segment map : segments) { + for (Segment map : segments) { map.clear(); } } @@ -179,12 +163,16 @@ public void setCapacity(long capacity) { } + @Override public void close() { + if (closed) { + return; + } closed = true; - for (Segment map : segments) { + for (Segment map : segments) { map.release(); } - Collections.fill(segments, null); + Arrays.fill(segments, null); if (logger.isDebugEnabled()) { logger.debug("Closing OHC instance"); @@ -195,17 +183,19 @@ public void close() { // statistics and related stuff // + @Override public void resetStatistics() { - for (Segment map : segments) { + for (Segment map : segments) { map.resetStatistics(); } putFailCount = 0; } + @Override public OffHeapHashTableStats stats() { long hitCount = 0, missCount = 0, size = 0, - freeCapacity = 0, rehashes = 0, putAddCount = 0, putReplaceCount = 0, removeCount = 0; - for (Segment map : segments) { + rehashes = 0, putAddCount = 0, putReplaceCount = 0, removeCount = 0; + for (Segment map : segments) { hitCount += map.hitCount(); missCount += map.missCount(); size += map.size(); @@ -227,51 +217,57 @@ public OffHeapHashTableStats stats() { perSegmentStats()); } + @Override public long size() { long size = 0L; - for (Segment map : segments) { + for (Segment map : segments) { size += map.size(); } return size; } + @Override public int segments() { - return segments.size(); + return segments.length; } + @Override public float loadFactor() { - return segments.get(0).loadFactor(); + return segments[0].loadFactor(); } + @Override public int[] hashTableSizes() { - int[] r = new int[segments.size()]; - for (int i = 0; i < segments.size(); i++) { - r[i] = segments.get(i).hashTableSize(); + int[] r = new int[segments.length]; + for (int i = 0; i < segments.length; i++) { + r[i] = segments[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(); + long[] r = new long[segments.length]; + for (int i = 0; i < segments.length; i++) { + r[i] = segments[i].size(); } return r; } + @Override public SegmentStats[] perSegmentStats() { - SegmentStats[] stats = new SegmentStats[segments.size()]; + SegmentStats[] stats = new SegmentStats[segments.length]; for (int i = 0; i < stats.length; i++) { - Segment map = segments.get(i); + Segment map = segments[i]; stats[i] = new SegmentStats(map.size(), map.numberOfChunks(), map.numberOfSlots(), map.freeListSize()); } return stats; } + @Override public EstimatedHistogram getBucketHistogram() { EstimatedHistogram hist = new EstimatedHistogram(); - for (Segment map : segments) { + for (Segment map : segments) { map.updateBucketHistogram(hist); } @@ -297,7 +293,8 @@ public EstimatedHistogram getBucketHistogram() { return new EstimatedHistogram(offsets, buckets); } + @Override public String toString() { - return getClass().getSimpleName() + " ,segments=" + segments.size(); + return getClass().getSimpleName() + " ,segments=" + segments.length; } } diff --git a/src/main/java/com/oath/halodb/Record.java b/src/main/java/com/oath/halodb/Record.java index dab7064..28c61fe 100644 --- a/src/main/java/com/oath/halodb/Record.java +++ b/src/main/java/com/oath/halodb/Record.java @@ -5,36 +5,17 @@ package com.oath.halodb; -import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.zip.CRC32; +import java.util.Objects; public class Record { - private final byte[] key, value; - private InMemoryIndexMetaData recordMetaData; - - private Header header; - public Record(byte[] key, byte[] value) { + Utils.validateKeySize(key.length); + Utils.validateValueSize(value.length); 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() { @@ -45,172 +26,21 @@ 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. - + public final boolean equals(Object obj) { + // final, all child classes only check key/value contents and are mutually compatible here 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; - } + @Override + public final int hashCode() { + return Objects.hash(Arrays.hashCode(key), Arrays.hashCode(value)); } } diff --git a/src/main/java/com/oath/halodb/RecordEntry.java b/src/main/java/com/oath/halodb/RecordEntry.java new file mode 100644 index 0000000..4bb91a7 --- /dev/null +++ b/src/main/java/com/oath/halodb/RecordEntry.java @@ -0,0 +1,188 @@ +/* + * 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; + +/** The internal record variant that represents an entry in the record file **/ +class RecordEntry extends Record { + + private final Header header; + + public RecordEntry(Header header, byte[] key, byte[] value) { + super(key, value); + this.header = header; + if (key.length != header.keySize) { + throw new IllegalArgumentException("Key size does not match header data. header: " + + header.keySize + " actual: " + key.length); + } + if (value.length != header.valueSize) { + throw new IllegalArgumentException("Value size does not match header data. header: " + + header.valueSize + " actual: " + value.length); + } + } + + /** create a RecordEntry from a Record, with a Header that has not yet computed its checksum **/ + static RecordEntry newEntry(Record record, long sequenceNumber) { + return newEntry(record.getKey(), record.getValue(), sequenceNumber); + } + + /** create a RecordEntry from a Record, with a Header that has not yet computed its checksum **/ + static RecordEntry newEntry(byte[] key, byte[] value, long sequenceNumber) { + Header header = new Header(0, Versions.CURRENT_DATA_FILE_VERSION, key.length, value.length, sequenceNumber); + return new RecordEntry(header, key, value); + } + + ByteBuffer[] serialize() { + ByteBuffer headerBuf = serializeHeaderAndComputeChecksum(); + return new ByteBuffer[] {headerBuf, ByteBuffer.wrap(getKey()), ByteBuffer.wrap(getValue())}; + } + + static RecordEntry deserialize(Header header, ByteBuffer buffer) { + buffer.flip(); + byte[] key = new byte[header.keySize]; + byte[] value = new byte[header.valueSize]; + buffer.get(key); + buffer.get(value); + return new RecordEntry(header, key, value); + } + + static int getValueOffset(int recordOffset, int keySize) { + return recordOffset + Header.HEADER_SIZE + keySize; + } + + static int getRecordSize(int keySize, int valueSize) { + return keySize + valueSize + Header.HEADER_SIZE; + } + + static int getValueSize(int recordSize, int keySize) { + return recordSize - Header.HEADER_SIZE - keySize; + } + + /** + * @return recordSize which is HEADER_SIZE + key size + value size. + */ + int getRecordSize() { + return header.getRecordSize(); + } + + long getSequenceNumber() { + return header.getSequenceNumber(); + } + + int getVersion() { + return header.version; + } + + Header getHeader() { + return 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(); + } + + 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(getKey()); + crc32.update(getValue()); + return crc32.getValue(); + } + + static class Header { + /** + * crc - 4 bytes. + * version + key size - 2 bytes. 5 bits for version, 11 for keySize + * 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 final long checkSum; + private final int valueSize; + private final long sequenceNumber; + private final int recordSize; + private final byte version; + private final short keySize; + + Header(long checkSum, byte version, int keySize, int valueSize, long sequenceNumber) { + this.checkSum = checkSum; + this.version = Utils.validateVersion(version); + this.keySize = Utils.validateKeySize(keySize); + this.valueSize = Utils.validateValueSize(valueSize); + this.sequenceNumber = Utils.validateSequenceNumber(sequenceNumber); + this.recordSize = keySize + valueSize + HEADER_SIZE; + } + + static Header deserialize(ByteBuffer buffer) { + + long checkSum = Utils.toUnsignedIntFromInt(buffer.getInt(CHECKSUM_OFFSET)); + byte vByte = buffer.get(VERSION_OFFSET); + byte keySizeByte = buffer.get(KEY_SIZE_OFFSET); + int valueSize = buffer.getInt(VALUE_SIZE_OFFSET); + long sequenceNumber = buffer.getLong(SEQUENCE_NUMBER_OFFSET); + byte version = Utils.version(vByte); + short keySize = Utils.keySize(vByte, keySizeByte); + + 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, Utils.versionByte(version, keySize)); + headerBuffer.put(KEY_SIZE_OFFSET, Utils.keySizeByte(keySize)); + headerBuffer.putInt(VALUE_SIZE_OFFSET, valueSize); + headerBuffer.putLong(SEQUENCE_NUMBER_OFFSET, sequenceNumber); + return headerBuffer; + } + + short getKeySize() { + return keySize; + } + + int getValueSize() { + return valueSize; + } + + int getRecordSize() { + return recordSize; + } + + long getSequenceNumber() { + return sequenceNumber; + } + + long getCheckSum() { + return checkSum; + } + + short getVersion() { + return version; + } + } +} diff --git a/src/main/java/com/oath/halodb/RecordIterated.java b/src/main/java/com/oath/halodb/RecordIterated.java new file mode 100644 index 0000000..9d09a51 --- /dev/null +++ b/src/main/java/com/oath/halodb/RecordIterated.java @@ -0,0 +1,20 @@ +/* + * 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; + +/** An internal Record variant that represents a record iterated over by HaloDb. **/ +class RecordIterated extends Record { + private final long sequenceNumber; + + RecordIterated(byte[] key, byte[] value, long sequenceNumber) { + super(key, value); + this.sequenceNumber = sequenceNumber; + } + + long getSequenceNumber() { + return sequenceNumber; + } +} diff --git a/src/main/java/com/oath/halodb/Segment.java b/src/main/java/com/oath/halodb/Segment.java index f636ca3..a198326 100644 --- a/src/main/java/com/oath/halodb/Segment.java +++ b/src/main/java/com/oath/halodb/Segment.java @@ -7,35 +7,29 @@ package com.oath.halodb; -import com.oath.halodb.histo.EstimatedHistogram; - import java.util.concurrent.atomic.AtomicLongFieldUpdater; -abstract class Segment { +import com.oath.halodb.histo.EstimatedHistogram; - final HashTableValueSerializer valueSerializer; - final int fixedValueLength; - final int fixedKeyLength; +abstract class Segment { - private final Hasher hasher; + final HashEntrySerializer serializer; + final int fixedKeyLength; 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(HashEntrySerializer entrySerializer) { + this(entrySerializer, -1); } - Segment(HashTableValueSerializer valueSerializer, int fixedValueLength, int fixedKeyLength, Hasher hasher) { - this.valueSerializer = valueSerializer; - this.fixedValueLength = fixedValueLength; + Segment(HashEntrySerializer serializer, int fixedKeyLength) { + this.serializer = serializer; this.fixedKeyLength = fixedKeyLength; - this.hasher = hasher; } - - boolean lock() { long t = Thread.currentThread().getId(); @@ -64,16 +58,11 @@ void unlock(boolean wasFirst) { assert r; } - KeyBuffer keySource(byte[] key) { - KeyBuffer keyBuffer = new KeyBuffer(key); - return keyBuffer.finish(hasher); - } - - abstract V getEntry(KeyBuffer key); + abstract E getEntry(KeyBuffer key); abstract boolean containsEntry(KeyBuffer key); - abstract boolean putEntry(byte[] key, V value, long hash, boolean ifAbsent, V oldValue); + abstract boolean putEntry(KeyBuffer key, E entry, boolean ifAbsent, E oldEntry); abstract boolean removeEntry(KeyBuffer key); diff --git a/src/main/java/com/oath/halodb/SegmentNonMemoryPool.java b/src/main/java/com/oath/halodb/SegmentNonMemoryPool.java index 81b6324..3b26278 100644 --- a/src/main/java/com/oath/halodb/SegmentNonMemoryPool.java +++ b/src/main/java/com/oath/halodb/SegmentNonMemoryPool.java @@ -7,18 +7,19 @@ 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 { +import com.google.common.primitives.Ints; +import com.oath.halodb.histo.EstimatedHistogram; + +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; + private static final int MAX_TABLE_POWER = 30; + private static final int MAX_TABLE_SIZE = 1 << MAX_TABLE_POWER; long size; Table table; @@ -39,8 +40,8 @@ class SegmentNonMemoryPool extends Segment { private static final boolean throwOOME = true; - SegmentNonMemoryPool(OffHeapHashTableBuilder builder) { - super(builder.getValueSerializer(), builder.getFixedValueSize(), builder.getHasher()); + SegmentNonMemoryPool(OffHeapHashTableBuilder builder) { + super(builder.getEntrySerializer()); this.hashAlgorithm = builder.getHashAlgorighm(); @@ -51,7 +52,7 @@ class SegmentNonMemoryPool extends Segment { if (hts < 256) { hts = 256; } - int msz = Ints.checkedCast(HashTableUtil.roundUpToPowerOf2(hts, MAX_TABLE_SIZE)); + int msz = HashTableUtil.roundUpToPowerOf2(hts, MAX_TABLE_POWER); table = Table.create(msz, throwOOME); if (table == null) { throw new RuntimeException("unable to allocate off-heap memory for segment"); @@ -123,16 +124,18 @@ long rehashes() { } @Override - V getEntry(KeyBuffer key) { + E getEntry(KeyBuffer key) { boolean wasFirst = lock(); try { for (long hashEntryAdr = table.getFirst(key.hash()); hashEntryAdr != 0L; hashEntryAdr = NonMemoryPoolHashEntries.getNext(hashEntryAdr)) { - if (key.sameKey(hashEntryAdr)) { + if (sameKey(key.buffer, hashEntryAdr, serializer)) { hitCount++; - return valueSerializer.deserialize(Uns.readOnlyBuffer(hashEntryAdr, fixedValueLength, NonMemoryPoolHashEntries.ENTRY_OFF_DATA + NonMemoryPoolHashEntries.getKeyLen(hashEntryAdr))); + long sizeAddress = hashEntryAdr + NonMemoryPoolHashEntries.ENTRY_OFF_DATA; + long locationAddress = sizeAddress + serializer.sizesSize(); + return serializer.deserialize(sizeAddress, locationAddress); } } @@ -150,7 +153,7 @@ boolean containsEntry(KeyBuffer key) { for (long hashEntryAdr = table.getFirst(key.hash()); hashEntryAdr != 0L; hashEntryAdr = NonMemoryPoolHashEntries.getNext(hashEntryAdr)) { - if (key.sameKey(hashEntryAdr)) { + if (sameKey(key.buffer, hashEntryAdr, serializer)) { hitCount++; return true; } @@ -164,40 +167,30 @@ boolean containsEntry(KeyBuffer key) { } @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; - } + boolean putEntry(KeyBuffer key, E entry, boolean ifAbsent, E oldEntry) { + byte[] keybytes = key.buffer; + long hash = key.hash(); + short keySize = Utils.validateKeySize(keybytes.length); + long hashEntryAdr = Uns.allocate(HashTableUtil.allocLen(keybytes.length, serializer.entrySize()), throwOOME); + if (hashEntryAdr == 0L) { + // entry too large to be inserted or OS is not able to provide enough memory + removeEntry(key); + return false; + } + // initialize hash entry + NonMemoryPoolHashEntries.init(hashEntryAdr); + serializeForPut(keybytes, entry, hashEntryAdr); + if (putEntry(hashEntryAdr, hash, keySize, ifAbsent, oldEntry)) { + return true; + } else { + // free if we did not insert the entry Uns.free(hashEntryAdr); return false; - } finally { - Uns.free(oldValueAdr); } } - private boolean putEntry(long newHashEntryAdr, long hash, long keyLen, boolean putIfAbsent, long oldValueAddr) { + private boolean putEntry(long newHashEntryAdr, long hash, short keyLen, boolean putIfAbsent, E oldEntry) { long removeHashEntryAdr = 0L; boolean wasFirst = lock(); try { @@ -206,7 +199,7 @@ private boolean putEntry(long newHashEntryAdr, long hash, long keyLen, boolean p for (hashEntryAdr = table.getFirst(hash); hashEntryAdr != 0L; prevEntryAdr = hashEntryAdr, hashEntryAdr = NonMemoryPoolHashEntries.getNext(hashEntryAdr)) { - if (notSameKey(newHashEntryAdr, hash, keyLen, hashEntryAdr)) { + if (notSameKey(newHashEntryAdr, keyLen, hashEntryAdr)) { continue; } @@ -215,10 +208,12 @@ private boolean putEntry(long newHashEntryAdr, long hash, long keyLen, boolean p 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)) { + // key already exists, we just need to replace the entry. + if (oldEntry != null) { + // if oldEntry does not match on replace(), don't insert + long sizeAddress = hashEntryAdr + NonMemoryPoolHashEntries.ENTRY_OFF_DATA; + long locationAddress = sizeAddress + serializer.sizesSize(); + if (!oldEntry.compare(sizeAddress, locationAddress)) { return false; } } @@ -232,9 +227,9 @@ private boolean putEntry(long newHashEntryAdr, long hash, long keyLen, boolean p // 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. + // key is not present but old entry is not null. // we consider this as a mismatch and return. - if (oldValueAddr != 0) { + if (oldEntry != null) { return false; } @@ -262,18 +257,22 @@ private boolean putEntry(long newHashEntryAdr, long hash, long keyLen, boolean p } } - private static boolean notSameKey(long newHashEntryAdr, long newHash, long newKeyLen, long hashEntryAdr) { - long serKeyLen = NonMemoryPoolHashEntries.getKeyLen(hashEntryAdr); + private boolean notSameKey(long newHashEntryAdr, short newKeyLen, long hashEntryAdr) { + short serKeyLen = keyLen(hashEntryAdr, serializer); return serKeyLen != newKeyLen - || !Uns.memoryCompare(hashEntryAdr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA, newHashEntryAdr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA, serKeyLen); + || !Uns.memoryCompare(hashEntryAdr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA + serializer.entrySize(), + newHashEntryAdr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA + serializer.entrySize(), + serKeyLen); } - private void serializeForPut(byte[] key, V value, long hashEntryAdr) { + private void serializeForPut(byte[] key, E entry, 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)); - } + // write index meta first, then key + long sizeAddress = hashEntryAdr + NonMemoryPoolHashEntries.ENTRY_OFF_DATA; + long locationAddress = sizeAddress + serializer.sizesSize(); + entry.serializeSizes(sizeAddress); + entry.serializeLocation(locationAddress); + Uns.copyMemory(key, 0, hashEntryAdr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA + serializer.entrySize(), key.length); } catch (Throwable e) { freeAndThrow(e, hashEntryAdr); } @@ -321,7 +320,7 @@ boolean removeEntry(KeyBuffer key) { for (long hashEntryAdr = table.getFirst(key.hash()); hashEntryAdr != 0L; prevEntryAdr = hashEntryAdr, hashEntryAdr = NonMemoryPoolHashEntries.getNext(hashEntryAdr)) { - if (!key.sameKey(hashEntryAdr)) { + if (!sameKey(key.buffer, hashEntryAdr, serializer)) { continue; } @@ -369,26 +368,30 @@ private void rehash() { next = NonMemoryPoolHashEntries.getNext(hashEntryAdr); NonMemoryPoolHashEntries.setNext(hashEntryAdr, 0L); - long hash = hasher.hash(hashEntryAdr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA, NonMemoryPoolHashEntries.getKeyLen(hashEntryAdr)); + short keySize = keyLen(hashEntryAdr, serializer); + long hash = hasher.hash(hashEntryAdr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA + serializer.entrySize(), keySize); newTable.addAsHead(hash, hashEntryAdr); } } - threshold = (long) ((float) newTable.size() * loadFactor); + threshold = (long) (newTable.size() * loadFactor); table.release(); table = newTable; rehashes++; logger.info("Completed rehashing segment in {} ms.", (System.currentTimeMillis() - start)); } + @Override float loadFactor() { return loadFactor; } + @Override int hashTableSize() { return table.size(); } + @Override void updateBucketHistogram(EstimatedHistogram hist) { boolean wasFirst = lock(); try { @@ -442,6 +445,7 @@ void release() { released = true; } + @Override protected void finalize() throws Throwable { if (!released) { Uns.free(address); @@ -528,4 +532,41 @@ private void add(long hashEntryAdr, long hash) { public String toString() { return String.valueOf(size); } + + static private short keyLen(long hashEntryAdr, HashEntrySerializer serializer) { + return serializer.readKeySize(hashEntryAdr + NonMemoryPoolHashEntries.ENTRY_OFF_DATA); + } + + static boolean sameKey(byte[] key, long hashEntryAdr, HashEntrySerializer serializer) { + return keyLen(hashEntryAdr, serializer) == key.length && compareKey(key, hashEntryAdr, serializer); + } + + + + static private boolean compareKey(byte[] key, long hashEntryAdr, HashEntrySerializer serializer) { + int blkOff = (int) NonMemoryPoolHashEntries.ENTRY_OFF_DATA + serializer.entrySize(); + int p = 0; + int endIdx = key.length; + for (; endIdx - p >= 8; p += 8) { + if (Uns.getLong(hashEntryAdr, blkOff + p) != Uns.getLongFromByteArray(key, p)) { + return false; + } + } + for (; endIdx - p >= 4; p += 4) { + if (Uns.getInt(hashEntryAdr, blkOff + p) != Uns.getIntFromByteArray(key, p)) { + return false; + } + } + for (; endIdx - p >= 2; p += 2) { + if (Uns.getShort(hashEntryAdr, blkOff + p) != Uns.getShortFromByteArray(key, p)) { + return false; + } + } + for (; endIdx - p >= 1; p += 1) { + if (Uns.getByte(hashEntryAdr, blkOff + p) != key[p]) { + return false; + } + } + return true; + } } diff --git a/src/main/java/com/oath/halodb/SegmentWithMemoryPool.java b/src/main/java/com/oath/halodb/SegmentWithMemoryPool.java index 9b1cebc..5831ba0 100644 --- a/src/main/java/com/oath/halodb/SegmentWithMemoryPool.java +++ b/src/main/java/com/oath/halodb/SegmentWithMemoryPool.java @@ -5,23 +5,23 @@ package com.oath.halodb; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.primitives.Ints; -import com.oath.halodb.histo.EstimatedHistogram; +import java.util.ArrayList; +import java.util.List; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.List; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.primitives.Ints; +import com.oath.halodb.histo.EstimatedHistogram; -class SegmentWithMemoryPool extends Segment { +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 static final int MAX_TABLE_POWER = 30; + private static final int MAX_TABLE_SIZE = 1 << MAX_TABLE_POWER; private long hitCount = 0; private long size = 0; @@ -33,35 +33,28 @@ class SegmentWithMemoryPool extends Segment { private final float loadFactor; private long rehashes = 0; - private final List chunks; - private byte currentChunkIndex = -1; + private final List> chunks = new ArrayList<>(); + private MemoryPoolChunk currentWriteChunk = null; private final int chunkSize; - private final MemoryPoolAddress emptyAddress = new MemoryPoolAddress((byte) -1, -1); - - private MemoryPoolAddress freeListHead = emptyAddress; + private MemoryPoolChunk.Slot freeListHead = null; private long freeListSize = 0; - private final int fixedSlotSize; + private final int slotSize; - private final HashTableValueSerializer valueSerializer; + private final HashEntrySerializer serializer; 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()); + SegmentWithMemoryPool(OffHeapHashTableBuilder builder) { + super(builder.getEntrySerializer(), builder.getFixedKeySize()); - this.chunks = new ArrayList<>(); this.chunkSize = builder.getMemoryPoolChunkSize(); - this.valueSerializer = builder.getValueSerializer(); - this.fixedSlotSize = MemoryPoolHashEntries.HEADER_SIZE + fixedKeyLength + fixedValueLength; + this.serializer = builder.getEntrySerializer(); + this.slotSize = MemoryPoolChunk.slotSize(fixedKeyLength, serializer); this.hashAlgorithm = builder.getHashAlgorighm(); int hts = builder.getHashTableSize(); @@ -71,8 +64,7 @@ class SegmentWithMemoryPool extends Segment { if (hts < 256) { hts = 256; } - int msz = Ints.checkedCast(HashTableUtil.roundUpToPowerOf2(hts, MAX_TABLE_SIZE)); - table = Table.create(msz); + table = Table.create(hts); if (table == null) { throw new RuntimeException("unable to allocate off-heap memory for segment"); } @@ -86,183 +78,260 @@ class SegmentWithMemoryPool extends Segment { } @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)); - } - } + public E getEntry(KeyBuffer key) { + return search(key, this::foundEntry, this::notFoundEntry); + } - missCount++; - return null; - } finally { - unlock(wasFirst); - } + private E foundEntry(MemoryPoolChunk.Slot head, MemoryPoolChunk.Slot previous, + MemoryPoolChunk.Slot tail, int chainLength) { + hitCount++; + return head.readEntry(); + } + + private E notFoundEntry(int slotAddress) { + missCount++; + return null; } @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; - } - } + return search(key, this::foundKey, this::notFoundKey); + } - missCount++; - return false; - } finally { - unlock(wasFirst); - } + private boolean foundKey(MemoryPoolChunk.Slot head, MemoryPoolChunk.Slot previous, + MemoryPoolChunk.Slot tail, int chainLength) { + hitCount++; + return true; } - @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); + private boolean notFoundKey(int slotAddress) { + missCount++; + return false; + } - 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. + @Override + boolean putEntry(KeyBuffer key, E entry, boolean dontOverwrite, E oldEntry) { + return search(key, + (head, previous, tail, chainLength) -> { + // key is already present in the segment. + + // putIfAbsent is true, but key is already present, return. + if (dontOverwrite) { + return false; + } - // putIfAbsent is true, but key is already present, return. - if (putIfAbsent) { + // code for replace() operation + if (oldEntry != null) { + if (!head.compareEntry(oldEntry)) { return false; } + } - // code for replace() operation - if (oldValue != null) { - if (!chunk.compareValue(address.chunkOffset, oldValueBuffer.array())) { - return false; - } - } + // replace value with the new one. + // if the key matches, we only have to modify the data in the head chunk that has + // the value data, extended key data remains in other chunks unaltered + head.setEntry(entry); + putReplaceCount++; + return true; + }, + (slotHead) -> { + // key is not present + if (oldEntry != null) { + // key is not present but old value is not null. + // we consider this as a mismatch and return. + return false; + } - // replace value with the new one. - chunk.setValue(newValueBuffer.array(), address.chunkOffset); - putReplaceCount++; - return true; + long hash = key.hash(); + if (size >= threshold) { + rehash(); + slotHead = table.getFirst(hash); } - } - if (oldValue != null) { - // key is not present but old value is not null. - // we consider this as a mismatch and return. - return false; - } + // key is not present in the segment, we need to add a new entry. + int nextSlotAddress = writeToFreeSlots(key.buffer, entry, slotHead).toAddress(); + table.addAsHead(hash, nextSlotAddress); + size++; + putAddCount++; + return true; + }); + } - 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); - } + @Override + public boolean removeEntry(KeyBuffer key) { + return search(key, + (head, previous, tail, chainLength) -> { + removeInternal(head, previous, tail, chainLength, key.hash()); + removeCount++; + size--; + return true; + }, (first) -> { + return false; + }); + } - return true; + @FunctionalInterface + private interface FoundEntryVisitor { + /** + * @param headChunk The chunk corresponding to the headOffset + * @param headOffset The offset into the headChunk for the first slot containing the key + * @param head The address of the first slot in the chain for the key + * @param previous The address of the slot in the list prior to the key, if it exists + * @param tail The last slot in the chain for this key, the same as the headAddress if the chain is size 1 + * @param chainLength The number of slots in the chain for the key + * @return The result that the search function will return when the key is found. + */ + A found(MemoryPoolChunk.Slot head, + MemoryPoolChunk.Slot previous, MemoryPoolChunk.Slot tail, int chainLength); } - @Override - public boolean removeEntry(KeyBuffer key) { + @FunctionalInterface + private interface NotFoundEntryVisitor { + /** + * + * @param firstAddress The first address for the slot corresponding to the hash of this key + * @return The result that the search function will return when the key is not found. + */ + A notFound(int firstAddress); + } + + private A search(KeyBuffer key, + FoundEntryVisitor whenFound, + NotFoundEntryVisitor whenNotFound) { 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; + MemoryPoolChunk.Slot previous = null; + int firstAddress = table.getFirst(key.hash()); + MemoryPoolChunk.Slot slot = slotFor(firstAddress); + while (slot != null) { + int ksize = key.buffer.length; + int slotKeySize = slot.getKeyLength(); + if (slotKeySize <= fixedKeyLength) { + // one slot, simple match and move on + if (slotKeySize == ksize && slot.compareFixedKey(key.buffer, ksize)) { + return whenFound.found(slot, previous, slot, 1); + } + } else { + // multiple slots, we must always traverse to the end of the chain for this key, even when it mismatches + int chainLength = 1; + MemoryPoolChunk.Slot headSlot = slot; + int remaining = slotKeySize - fixedKeyLength; + int maxFragmentSize = fixedKeyLength + serializer.entrySize(); + boolean fragmentMatches = slotKeySize == ksize && slot.compareFixedKey(key.buffer, fixedKeyLength); + do { + slot = slotFor(slot.getNextAddress()); + if (slot == null) { + throw new IllegalStateException("Corrupted slot state, extended key slot expected, found none"); + } + if (fragmentMatches) { + int compareOffset = ksize - remaining; + int compareLen = Math.min(maxFragmentSize, remaining); + fragmentMatches = slot.compareExtendedKey(key.buffer, compareOffset, compareLen); + chainLength++; + } + remaining -= maxFragmentSize; + } while (remaining > 0); + // we got through the key and all fragments matched, key found + if (fragmentMatches) { + return whenFound.found(headSlot, previous, slot, chainLength); + } } + previous = slot; + slot = slotFor(slot.getNextAddress()); } - - return false; + return whenNotFound.notFound(firstAddress); } 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()); + private MemoryPoolChunk.Slot slotFor(int poolAddress) { + if (MemoryPoolAddress.isEmpty(poolAddress)) { + return null; } + int chunkIndex = MemoryPoolAddress.chunkIndex(poolAddress); + int slot = MemoryPoolAddress.slot(poolAddress); + return chunkFor(chunkIndex).slotFor(slot); + } + + private MemoryPoolChunk chunkFor(int chunkIndex) { + if (chunkIndex < 1 || chunkIndex > chunks.size()) { + throw new IllegalArgumentException("Invalid chunk index " + chunkIndex + ". Chunk size " + chunks.size()); + } + return chunks.get(chunkIndex - 1); + } - MemoryPoolChunk chunk = chunks.get(address.chunkIndex); - return chunk.getNextAddress(address.chunkOffset); + private int getNext(int poolAddress) { + return slotFor(poolAddress).getNextAddress(); } - 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; + private MemoryPoolChunk.Slot writeToFreeSlots(byte[] key, E entry, int nextAddress) { + MemoryPoolChunk.Slot firstSlot = getFreeSlot(); + MemoryPoolChunk.Slot slot = firstSlot; + MemoryPoolChunk.Slot nextSlot = null; + int next; + if (key.length <= fixedKeyLength) { + next = nextAddress; + } else { + nextSlot = getFreeSlot(); + next = nextSlot.toAddress(); } - 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."); + slot.fillSlot(key, entry, next); + + int keyWritten = fixedKeyLength; + while (keyWritten < key.length) { + int keyRemaining = key.length - keyWritten; + int overflowSlotSpace = fixedKeyLength + serializer.entrySize(); + slot = nextSlot; + if (keyRemaining > overflowSlotSpace) { + nextSlot = getFreeSlot(); + slot.fillOverflowSlot(key, keyWritten, overflowSlotSpace, nextSlot.toAddress()); + } else { + slot.fillOverflowSlot(key, keyWritten, keyRemaining, nextAddress); } + keyWritten += overflowSlotSpace; + } + return firstSlot; + } + MemoryPoolChunk.Slot getFreeSlot() { + if (freeListHead != null) { + MemoryPoolChunk.Slot free = freeListHead; + freeListHead = slotFor(free.getNextAddress()); + freeListSize--; + return free; + } + if (currentWriteChunk == null || currentWriteChunk.isFull()) { + if (chunks.size() >= 255) { + logger.error("No more memory left. Each segment can have at most {} chunks.", 255); + throw new OutOfMemoryError("Each segment can have at most " + 255 + " 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; + // allocate an new one. + currentWriteChunk = MemoryPoolChunk.create(chunks.size() + 1, chunkSize, fixedKeyLength, serializer); + chunks.add(currentWriteChunk); } - - MemoryPoolChunk currentWriteChunk = chunks.get(currentChunkIndex); - MemoryPoolAddress slotAddress = new MemoryPoolAddress(currentChunkIndex, currentWriteChunk.getWriteOffset()); - currentWriteChunk.fillNextSlot(key, value, nextAddress); - return slotAddress; + return currentWriteChunk.allocateSlot(); } - private void removeInternal(MemoryPoolAddress address, MemoryPoolAddress previous, long hash) { - MemoryPoolAddress next = chunks.get(address.chunkIndex).getNextAddress(address.chunkOffset); - if (table.getFirst(hash).equals(address)) { + private void removeInternal(MemoryPoolChunk.Slot head, MemoryPoolChunk.Slot previous, + MemoryPoolChunk.Slot tail, int length, long hash) { + int next = tail.getNextAddress(); + if (previous == null) { 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); + previous.setNextAddress(next); } - chunks.get(address.chunkIndex).setNextAddress(address.chunkOffset, freeListHead); - freeListHead = address; - ++freeListSize; + if (freeListHead == null) { + tail.setNextAddress(MemoryPoolAddress.empty); + } else { + tail.setNextAddress(freeListHead.toAddress()); + } + freeListHead = head; + freeListSize += length; } private void rehash() { @@ -275,19 +344,48 @@ private void rehash() { 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); + + long hashBuffer = Uns.allocate(2048, true); // larger than max key size + try { + for (int i = 0; i < tableSize; i++) { + // each table slot is a chain of entries, individual keys can span more than one entry if the key + // size is larger than fixedKeyLength + int address = table.getFirst(i); + while (MemoryPoolAddress.nonEmpty(address)) { + int headAddress = address; + MemoryPoolChunk.Slot slot = slotFor(address); + int keySize = slot.getKeyLength(); + long hash; + if (keySize <= fixedKeyLength) { + // hash calculation is simple if the key fits in one slot + hash = slot.computeFixedKeyHash(hasher, keySize); + } else { + // otherwise, since hasher doesn't support incremental hashes, we have to copy the data to a buffer + // then hash + slot.copyEntireFixedKey(hashBuffer); + int copied = fixedKeyLength; + do { + address = slot.getNextAddress(); + slot = slotFor(address); + copied += slot.copyExtendedKey(hashBuffer, copied, keySize - copied); + } while (copied < keySize); + hash = hasher.hash(hashBuffer, 0, keySize); + } + // get the address the tail of this key points to + int next = slot.getNextAddress(); + int first = newTable.getFirst(hash); + // put the head of this key as the entry in the table + newTable.addAsHead(hash, headAddress); + // set the tail of this key to point to whatever was in the head of the new table + slot.setNextAddress(first); + address = next; + } } + } finally { + Uns.free(hashBuffer); } - threshold = (long) ((float) newTable.size() * loadFactor); + threshold = (long) (newTable.size() * loadFactor); table.release(); table = newTable; rehashes++; @@ -306,7 +404,7 @@ void release() { try { chunks.forEach(MemoryPoolChunk::destroy); chunks.clear(); - currentChunkIndex = -1; + currentWriteChunk = null; size = 0; table.release(); } finally { @@ -321,7 +419,7 @@ void clear() { try { chunks.forEach(MemoryPoolChunk::destroy); chunks.clear(); - currentChunkIndex = -1; + currentWriteChunk = null; size = 0; table.clear(); } finally { @@ -371,7 +469,7 @@ long numberOfChunks() { @Override long numberOfSlots() { - return chunks.size() * chunkSize / fixedSlotSize; + return chunks.size() * chunkSize / slotSize; } @Override @@ -398,7 +496,7 @@ int hashTableSize() { void updateBucketHistogram(EstimatedHistogram hist) { boolean wasFirst = lock(); try { - table.updateBucketHistogram(hist, chunks); + table.updateBucketHistogram(hist, this); } finally { unlock(wasFirst); } @@ -411,19 +509,21 @@ static final class Table { private boolean released; static Table create(int hashTableSize) { - int msz = Ints.checkedCast(HashTableUtil.MEMORY_POOL_BUCKET_ENTRY_LEN * hashTableSize); + int pow2Size = HashTableUtil.roundUpToPowerOf2(hashTableSize, MAX_TABLE_POWER); + + int msz = Ints.checkedCast(HashTableUtil.MEMORY_POOL_BUCKET_ENTRY_LEN * pow2Size); long address = Uns.allocate(msz, true); - return address != 0L ? new Table(address, hashTableSize) : null; + return address != 0L ? new Table(address, pow2Size) : null; } - private Table(long address, int hashTableSize) { + private Table(long address, int pow2Size) { this.address = address; - this.mask = hashTableSize - 1; + this.mask = pow2Size - 1; clear(); } void clear() { - Uns.setMemory(address, 0L, HashTableUtil.MEMORY_POOL_BUCKET_ENTRY_LEN * size(), (byte) -1); + Uns.setMemory(address, 0L, HashTableUtil.MEMORY_POOL_BUCKET_ENTRY_LEN * size(), (byte) 0); } void release() { @@ -431,6 +531,7 @@ void release() { released = true; } + @Override protected void finalize() throws Throwable { if (!released) { Uns.free(address); @@ -438,18 +539,14 @@ protected void finalize() throws Throwable { super.finalize(); } - MemoryPoolAddress getFirst(long hash) { + int 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); - + return Uns.getInt(bOffset, 0); } - void addAsHead(long hash, MemoryPoolAddress entryAddress) { + void addAsHead(long hash, int entryAddress) { long bOffset = address + bucketOffset(hash); - Uns.putByte(bOffset, 0, entryAddress.chunkIndex); - Uns.putInt(bOffset, 1, entryAddress.chunkOffset); + Uns.putInt(bOffset, 0, entryAddress); } long bucketOffset(long hash) { @@ -464,11 +561,10 @@ int size() { return mask + 1; } - void updateBucketHistogram(EstimatedHistogram h, final List chunks) { + void updateBucketHistogram(EstimatedHistogram h, final SegmentWithMemoryPool segment) { 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)) { + for (int adr = getFirst(i); !MemoryPoolAddress.isEmpty(adr); adr = segment.getNext(adr)) { len++; } h.add(len + 1); @@ -477,8 +573,12 @@ void updateBucketHistogram(EstimatedHistogram h, final List chu } @VisibleForTesting - MemoryPoolAddress getFreeListHead() { - return freeListHead; + int getFreeListHead() { + if (freeListHead == null) { + return MemoryPoolAddress.empty; + } else { + return freeListHead.toAddress(); + } } @VisibleForTesting diff --git a/src/main/java/com/oath/halodb/TombstoneEntry.java b/src/main/java/com/oath/halodb/TombstoneEntry.java index ae5b1e4..a6d85b8 100644 --- a/src/main/java/com/oath/halodb/TombstoneEntry.java +++ b/src/main/java/com/oath/halodb/TombstoneEntry.java @@ -9,13 +9,10 @@ import java.util.zip.CRC32; class TombstoneEntry { - //TODO: test. - /** - * crc - 4 byte - * version - 1 byte - * Key size - 1 byte - * Sequence number - 8 byte + * crc - 4 bytes. + * version + key size - 2 bytes. 5 bits for version, 11 for keySize, + * Sequence number - 8 bytes. */ static final int TOMBSTONE_ENTRY_HEADER_SIZE = 4 + 1 + 1 + 8; static final int CHECKSUM_SIZE = 4; @@ -28,9 +25,9 @@ class TombstoneEntry { private final byte[] key; private final long sequenceNumber; private final long checkSum; - private final int version; + private final byte version; - TombstoneEntry(byte[] key, long sequenceNumber, long checkSum, int version) { + TombstoneEntry(byte[] key, long sequenceNumber, long checkSum, byte version) { this.key = key; this.sequenceNumber = sequenceNumber; this.checkSum = checkSum; @@ -45,7 +42,7 @@ long getSequenceNumber() { return sequenceNumber; } - int getVersion() { + byte getVersion() { return version; } @@ -57,12 +54,16 @@ int size() { return TOMBSTONE_ENTRY_HEADER_SIZE + key.length; } - ByteBuffer[] serialize() { - byte keySize = (byte)key.length; + private ByteBuffer serializeHeader() { ByteBuffer header = ByteBuffer.allocate(TOMBSTONE_ENTRY_HEADER_SIZE); - header.put(VERSION_OFFSET, (byte)version); + header.put(VERSION_OFFSET, Utils.versionByte(version, key.length)); header.putLong(SEQUENCE_NUMBER_OFFSET, sequenceNumber); - header.put(KEY_SIZE_OFFSET, keySize); + header.put(KEY_SIZE_OFFSET, Utils.keySizeByte(key.length)); + return header; + } + + ByteBuffer[] serialize() { + ByteBuffer header = serializeHeader(); long crc32 = computeCheckSum(header.array()); header.putInt(CHECKSUM_OFFSET, Utils.toSignedIntFromLong(crc32)); return new ByteBuffer[] {header, ByteBuffer.wrap(key)}; @@ -70,26 +71,29 @@ ByteBuffer[] serialize() { static TombstoneEntry deserialize(ByteBuffer buffer) { long crc32 = Utils.toUnsignedIntFromInt(buffer.getInt()); - int version = Utils.toUnsignedByte(buffer.get()); + byte vByte = buffer.get(); long sequenceNumber = buffer.getLong(); - int keySize = (int)buffer.get(); + byte keySizeByte = buffer.get(); + byte version = Utils.version(vByte); + short keySize = Utils.keySize(vByte, keySizeByte); byte[] key = new byte[keySize]; buffer.get(key); return new TombstoneEntry(key, sequenceNumber, crc32, version); } - // returns null if a corrupted entry is detected. + // 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()); + byte vByte = buffer.get(); long sequenceNumber = buffer.getLong(); - int keySize = (int)buffer.get(); - if (sequenceNumber < 0 || keySize <= 0 || version < 0 || version > 255 || buffer.remaining() < keySize) + byte keySizeByte = buffer.get(); + byte version = Utils.version(vByte); + short keySize = Utils.keySize(vByte, keySizeByte); + if (sequenceNumber < 0 || keySize < 0 || version < 0 || version > 31 || buffer.remaining() < keySize) return null; byte[] key = new byte[keySize]; @@ -111,10 +115,7 @@ private long computeCheckSum(byte[] header) { } 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); + ByteBuffer header = serializeHeader(); return computeCheckSum(header.array()); } } diff --git a/src/main/java/com/oath/halodb/TombstoneFile.java b/src/main/java/com/oath/halodb/TombstoneFile.java index 27cac1c..439c715 100644 --- a/src/main/java/com/oath/halodb/TombstoneFile.java +++ b/src/main/java/com/oath/halodb/TombstoneFile.java @@ -5,8 +5,8 @@ package com.oath.halodb; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; +import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; import java.io.File; import java.io.IOException; @@ -18,13 +18,14 @@ import java.util.Iterator; import java.util.Objects; -import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; -import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class TombstoneFile { private static final Logger logger = LoggerFactory.getLogger(TombstoneFile.class); private final File backingFile; + private RandomAccessFile raf; private FileChannel channel; private final DBDirectory dbDirectory; @@ -58,12 +59,13 @@ static TombstoneFile create(DBDirectory dbDirectory, int fileId, HaloDBOptions o } void open() throws IOException { - channel = new RandomAccessFile(backingFile, "rw").getChannel(); + raf = new RandomAccessFile(backingFile, "rw"); + channel = raf.getChannel(); } void close() throws IOException { - if (channel != null) { - channel.close(); + if (raf != null) { + raf.close(); } } @@ -129,7 +131,7 @@ TombstoneFile repairFile(DBDirectory dbDirectory) throws IOException { 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(); + dbDirectory.syncMetaData(); repairFile.close(); close(); open(); @@ -193,7 +195,7 @@ public TombstoneEntry next() { if (hasNext()) { if (discardCorruptedRecords) return TombstoneEntry.deserializeIfNotCorrupted(buffer); - + return TombstoneEntry.deserialize(buffer); } diff --git a/src/main/java/com/oath/halodb/Uns.java b/src/main/java/com/oath/halodb/Uns.java index a22b39a..abbe45b 100644 --- a/src/main/java/com/oath/halodb/Uns.java +++ b/src/main/java/com/oath/halodb/Uns.java @@ -7,11 +7,6 @@ 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; @@ -22,6 +17,11 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import sun.misc.Unsafe; + final class Uns { private static final Logger LOGGER = LoggerFactory.getLogger(Uns.class); @@ -35,8 +35,8 @@ final class Uns { // // #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 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 { @@ -301,6 +301,31 @@ static boolean memoryCompare(long adr1, long off1, long adr2, long off2, long le return true; } + static boolean compare(long address, byte[] array, int arrayoffset, int len) { + int p = 0, length = len; + for (; length - p >= 8; p += 8) { + if (Uns.getLong(address, p) != Uns.getLongFromByteArray(array, p + arrayoffset)) { + return false; + } + } + for (; length - p >= 4; p += 4) { + if (Uns.getInt(address, p) != Uns.getIntFromByteArray(array, p + arrayoffset)) { + return false; + } + } + for (; length - p >= 2; p += 2) { + if (Uns.getShort(address, p) != Uns.getShortFromByteArray(array, p + arrayoffset)) { + return false; + } + } + for (; length - p >= 1; p += 1) { + if (Uns.getByte(address, p) != array[p + arrayoffset]) { + return false; + } + } + return true; + } + static long crc32(long address, long offset, long len) { validate(address, offset, len); return ext.crc32(address, offset, len); diff --git a/src/main/java/com/oath/halodb/Utils.java b/src/main/java/com/oath/halodb/Utils.java index 49fb021..4af39fe 100644 --- a/src/main/java/com/oath/halodb/Utils.java +++ b/src/main/java/com/oath/halodb/Utils.java @@ -10,32 +10,65 @@ 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; + static long toUnsignedIntFromInt(int value) { + return value & 0xffffffffL; } - //TODO: probably belongs to Record. - static int getRecordSize(int keySize, int valueSize) { - return keySize + valueSize + Record.Header.HEADER_SIZE; + static int toSignedIntFromLong(long value) { + return (int)(value & 0xffffffffL); } - static int getValueSize(int recordSize, byte[] key) { - return recordSize - Record.Header.HEADER_SIZE - key.length; + static int toUnsignedByte(byte value) { + return value & 0xFF; } - 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()); + /* max 31 */ + static byte version(byte versionByte) { + return (byte) (versionByte >>> 3); // 5 most significant bits } - static long toUnsignedIntFromInt(int value) { - return value & 0xffffffffL; + /* max 2047 */ + static short keySize(byte versionByte, byte keySizeByte) { + int upper = (versionByte & 0b111) << 8; // lowest three bits of version byte are 3 MSB of keySize + int lower = 0xFF & keySizeByte; + return (short) (upper | lower); } - static int toSignedIntFromLong(long value) { - return (int)(value & 0xffffffffL); + static byte versionByte(byte version, int keySize) { + validateVersion(version); + validateKeySize(keySize); + return (byte)(((version << 3) | (keySize >>> 8)) & 0xFF); } - static int toUnsignedByte(byte value) { - return value & 0xFF; + static byte keySizeByte(int keySize) { + validateKeySize(keySize); + return (byte)(keySize & 0xFF); + } + static byte validateVersion(byte version) { + if ((version >>> 5) != 0) { + throw new IllegalArgumentException("Version must be between 0 and 31, but was: " + version); + } + return version; + } + + static short validateKeySize(int keySize) { + if ((keySize >>> 11) != 0) { + throw new IllegalArgumentException("Key size must be between 0 and 2047, but was: " + keySize); + } + return (short) (keySize & 0xFFFF); + } + + static int validateValueSize(int valueSize) { + if ((valueSize >>> 29) != 0) { + throw new IllegalArgumentException("Value size must be between 0 and 536870912 (~512MB), but was: " + valueSize); + } + return valueSize; + } + + static long validateSequenceNumber(long sequenceNumber) { + if (sequenceNumber < 0) { + throw new IllegalArgumentException("Sequence number must be positive, but was: " + sequenceNumber); + } + return sequenceNumber; } } diff --git a/src/main/java/com/oath/halodb/Versions.java b/src/main/java/com/oath/halodb/Versions.java index 3c55188..52a370b 100644 --- a/src/main/java/com/oath/halodb/Versions.java +++ b/src/main/java/com/oath/halodb/Versions.java @@ -7,8 +7,8 @@ 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; + static final byte CURRENT_DATA_FILE_VERSION = 0; + static final byte CURRENT_INDEX_FILE_VERSION = 0; + static final byte CURRENT_TOMBSTONE_FILE_VERSION = 0; + static final byte CURRENT_META_FILE_VERSION = 0; } diff --git a/src/test/java/com/oath/halodb/ByteArrayEntry.java b/src/test/java/com/oath/halodb/ByteArrayEntry.java new file mode 100644 index 0000000..11be544 --- /dev/null +++ b/src/test/java/com/oath/halodb/ByteArrayEntry.java @@ -0,0 +1,50 @@ +package com.oath.halodb; + +import java.util.Arrays; + +class ByteArrayEntry extends HashEntry { + private final boolean failOnSerialize; + final byte[] bytes; + + ByteArrayEntry(int keySize, byte[] bytes) { + this(keySize, bytes, false); + } + + ByteArrayEntry(int keySize, byte[] bytes, boolean failOnSerialize) { + super(keySize, bytes.length); + this.failOnSerialize = failOnSerialize; + this.bytes = bytes; + } + + @Override + public int hashCode() { + return (31 * getKeySize()) + Arrays.hashCode(bytes); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + ByteArrayEntry other = (ByteArrayEntry) obj; + return getKeySize() == other.getKeySize() && Arrays.equals(bytes, other.bytes); + } + + @Override + void serializeLocation(long locationAddress) { + if (failOnSerialize) { + throw new RuntimeException("boom"); + } + Uns.copyMemory(bytes, 0, locationAddress, 0, bytes.length); + } + + @Override + boolean compareLocation(long locationAddress) { + byte[] data = new byte[getValueSize()]; + Uns.copyMemory(locationAddress, 0, data, 0, data.length); + return Arrays.equals(bytes, data); + } +} \ No newline at end of file diff --git a/src/test/java/com/oath/halodb/ByteArrayEntrySerializer.java b/src/test/java/com/oath/halodb/ByteArrayEntrySerializer.java new file mode 100644 index 0000000..cb237e0 --- /dev/null +++ b/src/test/java/com/oath/halodb/ByteArrayEntrySerializer.java @@ -0,0 +1,51 @@ +package com.oath.halodb; + +class ByteArrayEntrySerializer extends HashEntrySerializer { + + private final int arraySize; + + ByteArrayEntrySerializer(int arraySize) { + this.arraySize = arraySize; + } + + @Override + ByteArrayEntry deserialize(long sizeAddress, long locationAddress) { + int firstWord = Uns.getInt(sizeAddress, 0); + byte nextByte = Uns.getByte(sizeAddress, 4); + short keySize = HashEntry.extractKeySize(firstWord); + int valueSize = HashEntry.extractValueSize(firstWord, nextByte); + validateArraySize(valueSize); + byte[] bytes = new byte[valueSize]; + Uns.copyMemory(locationAddress, 0, bytes, 0, arraySize); + return new ByteArrayEntry(keySize, bytes); + } + + @Override + int locationSize() { + return arraySize; + } + + @Override + boolean validSize(ByteArrayEntry entry) { + return entry.bytes.length == arraySize; + } + + static ByteArrayEntrySerializer ofSize(int size) { + return new ByteArrayEntrySerializer(size); + } + + ByteArrayEntry randomEntry(int keySize) { + return new ByteArrayEntry(keySize, HashTableTestUtils.randomBytes(arraySize)); + } + + ByteArrayEntry createEntry(int keySize, byte[] bytes) { + validateArraySize(bytes.length); + return new ByteArrayEntry(keySize, bytes); + } + + private void validateArraySize(int length) { + if (length != arraySize) { + throw new IllegalArgumentException("invalid entry size, expected" + arraySize + " but was " + length); + } + } +} diff --git a/src/test/java/com/oath/halodb/ByteArrayEntrySerializerTest.java b/src/test/java/com/oath/halodb/ByteArrayEntrySerializerTest.java new file mode 100644 index 0000000..a47d105 --- /dev/null +++ b/src/test/java/com/oath/halodb/ByteArrayEntrySerializerTest.java @@ -0,0 +1,49 @@ +/* + * 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; + +public class ByteArrayEntrySerializerTest implements HashEntrySerializerTest { + + ByteArrayEntrySerializer serializer = ByteArrayEntrySerializer.ofSize(4); + short ksize = 1111; + byte[] data = new byte[] { 0, 1, 2, 3 }; + byte[] data2 = new byte[] { 3, 2, 1, 0 }; + byte[] tooLarge = new byte[] { 0, 1, 2, 3, 4 }; + + @Test + public void testSerializeDeserialize() { + + ByteArrayEntry entry = serializer.createEntry(ksize, data); + Assert.assertEquals(entry.getKeySize(), ksize); + Assert.assertEquals(data, entry.bytes); + + ByteArrayEntry entry2 = serializer.createEntry(ksize, data2); + Assert.assertNotEquals(entry2, entry); + ByteArrayEntry entry3 = serializer.createEntry(1, data); + Assert.assertNotEquals(entry3, entry); + Assert.assertEquals(entry, entry); + + Assert.assertFalse(entry.equals(new Object())); + Assert.assertFalse(entry.equals(null)); + + ByteArrayEntry readEntry = testSerDe(entry, serializer, (e1, e2) -> e2.equals(e1)); + Assert.assertEquals(readEntry.hashCode(), entry.hashCode()); + } + + @Test(expectedExceptions = IllegalArgumentException.class) + public void testInvalidSize() { + serializer.createEntry(ksize, tooLarge); + } + + @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "boom") + public void testSerializationFailure() { + testSerDe(new ByteArrayEntry(333, data2, true), serializer, (e1, e2) -> e2.equals(e1)); + } + +} diff --git a/src/test/java/com/oath/halodb/CheckOffHeapHashTable.java b/src/test/java/com/oath/halodb/CheckOffHeapHashTable.java index 5c79cfb..5d1f1c6 100644 --- a/src/test/java/com/oath/halodb/CheckOffHeapHashTable.java +++ b/src/test/java/com/oath/halodb/CheckOffHeapHashTable.java @@ -9,16 +9,13 @@ 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 +final class CheckOffHeapHashTable implements OffHeapHashTable { - private final HashTableValueSerializer valueSerializer; + private final HashEntrySerializer serializer; private final CheckSegment[] maps; private final int segmentShift; @@ -27,7 +24,7 @@ final class CheckOffHeapHashTable implements OffHeapHashTable private long putFailCount; private final Hasher hasher; - CheckOffHeapHashTable(OffHeapHashTableBuilder builder) + CheckOffHeapHashTable(OffHeapHashTableBuilder builder) { loadFactor = builder.getLoadFactor(); hasher = Hasher.create(builder.getHashAlgorighm()); @@ -41,47 +38,51 @@ final class CheckOffHeapHashTable implements OffHeapHashTable for (int i = 0; i < maps.length; i++) maps[i] = new CheckSegment(builder.getHashTableSize(), builder.getLoadFactor()); - valueSerializer = builder.getValueSerializer(); + serializer = builder.getEntrySerializer(); } - public boolean put(byte[] key, V value) + @Override + public boolean put(byte[] key, E entry) { KeyBuffer keyBuffer = keySource(key); - byte[] data = value(value); + byte[] data = entry(entry); CheckSegment segment = segment(keyBuffer.hash()); return segment.put(keyBuffer, data, false, null); } - public boolean addOrReplace(byte[] key, V old, V value) + @Override + public boolean addOrReplace(byte[] key, E old, E entry) { KeyBuffer keyBuffer = keySource(key); - byte[] data = value(value); - byte[] oldData = value(old); + byte[] data = entry(entry); + byte[] oldData = entry(old); CheckSegment segment = segment(keyBuffer.hash()); return segment.put(keyBuffer, data, false, oldData); } - public boolean putIfAbsent(byte[] key, V v) + @Override + public boolean putIfAbsent(byte[] key, E v) { KeyBuffer keyBuffer = keySource(key); - byte[] data = value(v); + byte[] data = entry(v); CheckSegment segment = segment(keyBuffer.hash()); return segment.put(keyBuffer, data, true, null); } - public boolean putIfAbsent(byte[] key, V value, long expireAt) + public boolean putIfAbsent(byte[] key, E entry, long expireAt) { throw new UnsupportedOperationException(); } - public boolean put(byte[] key, V value, long expireAt) + public boolean put(byte[] key, E entry, long expireAt) { throw new UnsupportedOperationException(); } + @Override public boolean remove(byte[] key) { KeyBuffer keyBuffer = keySource(key); @@ -89,24 +90,34 @@ public boolean remove(byte[] key) return segment.remove(keyBuffer); } + @Override public void clear() { for (CheckSegment map : maps) map.clear(); } - public V get(byte[] key) + @Override + public E get(byte[] key) { KeyBuffer keyBuffer = keySource(key); CheckSegment segment = segment(keyBuffer.hash()); - byte[] value = segment.get(keyBuffer); + byte[] entry = segment.get(keyBuffer); - if (value == null) + if (entry == null) { return null; - - return valueSerializer.deserialize(ByteBuffer.wrap(value)); + } + int entryLen = serializer.entrySize(); + long adr = Uns.allocate(entryLen); + try { + Uns.copyMemory(entry, 0, adr, 0, entryLen); + return serializer.deserialize(adr, adr + serializer.sizesSize()); + } finally { + Uns.free(adr); + } } + @Override public boolean containsKey(byte[] key) { KeyBuffer keyBuffer = keySource(key); @@ -114,6 +125,7 @@ public boolean containsKey(byte[] key) return segment.get(keyBuffer) != null; } + @Override public void resetStatistics() { for (CheckSegment map : maps) @@ -121,6 +133,7 @@ public void resetStatistics() putFailCount = 0; } + @Override public long size() { long r = 0; @@ -129,12 +142,14 @@ public long size() return r; } + @Override public int[] hashTableSizes() { // no hash table size info return new int[maps.length]; } + @Override public SegmentStats[] perSegmentStats() { SegmentStats[] stats = new SegmentStats[maps.length]; for (int i = 0; i < stats.length; i++) { @@ -145,21 +160,25 @@ public SegmentStats[] perSegmentStats() { return stats; } + @Override public EstimatedHistogram getBucketHistogram() { throw new UnsupportedOperationException(); } + @Override public int segments() { return maps.length; } + @Override public float loadFactor() { return loadFactor; } + @Override public OffHeapHashTableStats stats() { return new OffHeapHashTableStats( @@ -215,6 +234,7 @@ private long missCount() return missCount; } + @Override public void close() { clear(); @@ -235,13 +255,21 @@ KeyBuffer keySource(byte[] key) { return keyBuffer.finish(hasher); } - private byte[] value(V value) + private byte[] entry(E entry) { - if (value == null) { + if (entry == null) { return null; } - ByteBuffer buf = ByteBuffer.allocate(valueSerializer.serializedSize(value)); - valueSerializer.serialize(value, buf); - return buf.array(); + int entryLen = serializer.entrySize(); + long adr = Uns.allocate(entryLen); + try { + entry.serializeSizes(adr); + entry.serializeLocation(adr + serializer.sizesSize()); + byte[] out = new byte[entryLen]; + Uns.copyMemory(adr, 0, out, 0, entryLen); + return out; + } finally { + Uns.free(adr); + } } } diff --git a/src/test/java/com/oath/halodb/CheckSegment.java b/src/test/java/com/oath/halodb/CheckSegment.java index dc2cf45..cae42c0 100644 --- a/src/test/java/com/oath/halodb/CheckSegment.java +++ b/src/test/java/com/oath/halodb/CheckSegment.java @@ -11,7 +11,6 @@ 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}. diff --git a/src/test/java/com/oath/halodb/CompactionWithErrorsTest.java b/src/test/java/com/oath/halodb/CompactionWithErrorsTest.java index 7390a80..eabc770 100644 --- a/src/test/java/com/oath/halodb/CompactionWithErrorsTest.java +++ b/src/test/java/com/oath/halodb/CompactionWithErrorsTest.java @@ -5,20 +5,15 @@ package com.oath.halodb; -import com.google.common.util.concurrent.RateLimiter; +import java.io.IOException; +import java.util.List; 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 com.google.common.util.concurrent.RateLimiter; import mockit.Expectations; -import mockit.Invocation; import mockit.Mock; import mockit.MockUp; import mockit.Mocked; @@ -35,7 +30,7 @@ public void testCompactionWithException() throws HaloDBException, InterruptedExc @Mock public double acquire(int permits) { if (++callCount == 3) { - // throw an exception when copying the third record. + // throw an exception when copying the third record. throw new OutOfMemoryError("Throwing mock exception form compaction thread."); } return 10; @@ -56,7 +51,7 @@ public double acquire(int permits) { TestUtils.waitForCompactionToComplete(db); // An exception was thrown while copying a record in the compaction thread. - // Make sure that all records are still correct. + // 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()); @@ -67,7 +62,7 @@ public double acquire(int permits) { // 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. + // 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()); @@ -88,7 +83,7 @@ public void testRestartCompactionThreadAfterCrash(@Mocked CompactionManager comp @Mock public double acquire(int permits) { if (++callCount == 3 || callCount == 8) { - // throw exceptions twice, each time compaction thread should crash and restart. + // throw exceptions twice, each time compaction thread should crash and restart. throw new OutOfMemoryError("Throwing mock exception from compaction thread."); } return 10; @@ -130,14 +125,14 @@ public double acquire(int permits) { // called when db.open() compactionManager.startCompactionThread(); - // compaction thread should have crashed twice and each time it should have been restarted. + // 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. + // called when db.open() the second time. compactionManager.startCompactionThread(); }}; @@ -176,12 +171,12 @@ boolean stopCompactionThread(boolean flag) throws IOException { DBMetaData dbMetaData = new DBMetaData(dbDirectory); dbMetaData.loadFromFileIfExists(); - // Since there was an IOException while stopping compaction IOError flag must have been set. + // 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); + List records = TestUtils.insertRandomRecordsOfSize(db, numberOfRecords, 1024 - RecordEntry.Header.HEADER_SIZE); // Update first 5 records in each file. for (int i = 0; i < 5; i++) { diff --git a/src/test/java/com/oath/halodb/CrossCheckTest.java b/src/test/java/com/oath/halodb/CrossCheckTest.java index ce5a6bd..dad3cb1 100644 --- a/src/test/java/com/oath/halodb/CrossCheckTest.java +++ b/src/test/java/com/oath/halodb/CrossCheckTest.java @@ -7,13 +7,8 @@ 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 static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; @@ -22,16 +17,23 @@ import java.util.Map; import java.util.Random; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertTrue; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import com.google.common.primitives.Longs; +import com.oath.halodb.HashTableTestUtils.KeyEntryPair; +import com.oath.halodb.histo.EstimatedHistogram; // 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; + private static final ByteArrayEntrySerializer serializer = ByteArrayEntrySerializer.ofSize(14); + private static final ByteArrayEntrySerializer bigSerializer = ByteArrayEntrySerializer.ofSize(15); @AfterMethod(alwaysRun = true) public void deinit() @@ -39,27 +41,25 @@ public void deinit() Uns.clearUnsDebugForTest(); } - static DoubleCheckOffHeapHashTableImpl cache(HashAlgorithm hashAlgorithm, boolean useMemoryPool) + static DoubleCheckOffHeapHashTableImpl cache(HashAlgorithm hashAlgorithm, boolean useMemoryPool) { return cache(hashAlgorithm, useMemoryPool, 256); } - static DoubleCheckOffHeapHashTableImpl cache(HashAlgorithm hashAlgorithm, boolean useMemoryPool, long capacity) + 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) + 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) + 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); + OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(serializer) + .hashMode(hashAlgorithm); if (useMemoryPool) builder.useMemoryPool(true).fixedKeySize(fixedKeySize); @@ -91,22 +91,22 @@ public Object[][] cacheEviction() @Test(dataProvider = "hashAlgorithms") public void testBasics(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws IOException, InterruptedException { - try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) + try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) { - byte[] key = HashTableTestUtils.randomBytes(12); - byte[] value = HashTableTestUtils.randomBytes(fixedValueSize); + byte[] key = HashTableTestUtils.randomBytesOfRange(fixedKeySize / 2, fixedKeySize, fixedKeySize * 8); + ByteArrayEntry value = serializer.randomEntry(key.length); cache.put(key, value); - byte[] actual = cache.get(key); + ByteArrayEntry actual = cache.get(key); Assert.assertEquals(actual, value); cache.remove(key); - Map keyValues = new HashMap<>(); + Map keyValues = new HashMap<>(); for (int i = 0; i < 100; i++) { - byte[] k = HashTableTestUtils.randomBytes(8); - byte[] v = HashTableTestUtils.randomBytes(fixedValueSize); + byte[] k = HashTableTestUtils.randomBytesOfRange(fixedKeySize / 2, fixedKeySize, fixedKeySize * 8); + ByteArrayEntry v = serializer.randomEntry(k.length); keyValues.put(k, v); cache.put(k, v); } @@ -123,15 +123,15 @@ public void testBasics(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throw @Test(dataProvider = "hashAlgorithms", dependsOnMethods = "testBasics") public void testManyValues(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws IOException, InterruptedException { - try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool, 64, -1)) + try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool, 64, -1)) { - List entries = HashTableTestUtils.fillMany(cache, fixedValueSize); + List entries = HashTableTestUtils.fillMany(cache, serializer); 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)); + entries.forEach(kv -> Assert.assertEquals(cache.get(kv.key), kv.entry)); stats = cache.stats(); Assert.assertEquals(stats.getHitCount(), HashTableTestUtils.manyCount); @@ -139,12 +139,12 @@ public void testManyValues(HashAlgorithm hashAlgorithm, boolean useMemoryPool) t 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); + KeyEntryPair kv = entries.get(i); + Assert.assertEquals(cache.get(kv.key), kv.entry, "for i="+i); assertTrue(cache.containsKey(kv.key), "for i="+i); - byte[] updated = HashTableTestUtils.randomBytes(fixedValueSize); + ByteArrayEntry updated = serializer.randomEntry(kv.key.length); cache.put(kv.key, updated); - entries.set(i, new HashTableTestUtils.KeyValuePair(kv.key, updated)); + entries.set(i, new KeyEntryPair(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); @@ -154,7 +154,7 @@ public void testManyValues(HashAlgorithm hashAlgorithm, boolean useMemoryPool) t Assert.assertEquals(stats.getPutReplaceCount(), HashTableTestUtils.manyCount); Assert.assertEquals(stats.getSize(), HashTableTestUtils.manyCount); - entries.forEach(kv -> Assert.assertEquals(cache.get(kv.key), kv.value)); + entries.forEach(kv -> Assert.assertEquals(cache.get(kv.key), kv.entry)); stats = cache.stats(); Assert.assertEquals(stats.getHitCount(), HashTableTestUtils.manyCount * 6); @@ -162,8 +162,8 @@ public void testManyValues(HashAlgorithm hashAlgorithm, boolean useMemoryPool) t 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); + KeyEntryPair kv = entries.get(i); + Assert.assertEquals(cache.get(kv.key), kv.entry, "for i=" + i); assertTrue(cache.containsKey(kv.key), "for i=" + i); cache.remove(kv.key); Assert.assertNull(cache.get(kv.key), "for i=" + i); @@ -183,10 +183,8 @@ public void testManyValues(HashAlgorithm hashAlgorithm, boolean useMemoryPool) t public void testRehash(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws IOException, InterruptedException { int count = 10_000; - OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder() - .valueSerializer(HashTableTestUtils.byteArraySerializer) + OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(serializer) .hashMode(hashAlgorithm) - .fixedValueSize(fixedValueSize) .hashTableSize(count/4) .segmentCount(1) .loadFactor(1); @@ -195,16 +193,16 @@ public void testRehash(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throw builder.useMemoryPool(true).fixedKeySize(fixedKeySize); - try (OffHeapHashTable cache = new DoubleCheckOffHeapHashTableImpl<>(builder)) + try (OffHeapHashTable cache = new DoubleCheckOffHeapHashTableImpl<>(builder)) { - List entries = HashTableTestUtils.fill(cache, fixedValueSize, count); + List entries = HashTableTestUtils.fill(cache, serializer, 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)); + entries.forEach(kv -> Assert.assertEquals(cache.get(kv.key), kv.entry)); stats = cache.stats(); Assert.assertEquals(stats.getHitCount(), count); @@ -212,12 +210,12 @@ public void testRehash(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throw for (int i = 0; i < count; i++) { - HashTableTestUtils.KeyValuePair kv = entries.get(i); - Assert.assertEquals(cache.get(kv.key), kv.value, "for i="+i); + KeyEntryPair kv = entries.get(i); + Assert.assertEquals(cache.get(kv.key), kv.entry, "for i="+i); assertTrue(cache.containsKey(kv.key), "for i="+i); - byte[] updated = HashTableTestUtils.randomBytes(fixedValueSize); + ByteArrayEntry updated = serializer.randomEntry(kv.key.length); cache.put(kv.key, updated); - entries.set(i, new HashTableTestUtils.KeyValuePair(kv.key, updated)); + entries.set(i, new KeyEntryPair(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); @@ -228,7 +226,7 @@ public void testRehash(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throw Assert.assertEquals(stats.getSize(), count); Assert.assertEquals(stats.getRehashCount(), 2); - entries.forEach(kv -> Assert.assertEquals(cache.get(kv.key), kv.value)); + entries.forEach(kv -> Assert.assertEquals(cache.get(kv.key), kv.entry)); stats = cache.stats(); Assert.assertEquals(stats.getHitCount(), count * 6); @@ -236,8 +234,8 @@ public void testRehash(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throw for (int i = 0; i < count; i++) { - HashTableTestUtils.KeyValuePair kv = entries.get(i); - Assert.assertEquals(cache.get(kv.key), kv.value, "for i=" + i); + KeyEntryPair kv = entries.get(i); + Assert.assertEquals(cache.get(kv.key), kv.entry, "for i=" + i); assertTrue(cache.containsKey(kv.key), "for i=" + i); cache.remove(kv.key); Assert.assertNull(cache.get(kv.key), "for i=" + i); @@ -253,39 +251,27 @@ public void testRehash(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throw } } - - -// -// 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.*") + expectedExceptionsMessageRegExp = ".*value size incompatible with fixed value size.*") public void testPutTooLargeValue(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws IOException, InterruptedException { byte[] key = HashTableTestUtils.randomBytes(8); - byte[] largeValue = HashTableTestUtils.randomBytes(fixedValueSize + 1); + ByteArrayEntry largeEntry = bigSerializer.randomEntry(key.length); - try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool, 1, -1)) { - cache.put(key, largeValue); + try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool, 1, -1)) { + cache.put(key, largeEntry); } } @Test(dataProvider = "hashAlgorithms", dependsOnMethods = "testBasics", expectedExceptions = IllegalArgumentException.class, - expectedExceptionsMessageRegExp = ".*exceeds max permitted size of 127") + expectedExceptionsMessageRegExp = "Key size must be between 0 and 2047, but was: 2048") public void testPutTooLargeKey(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws IOException, InterruptedException { - byte[] key = HashTableTestUtils.randomBytes(1024); - byte[] largeValue = HashTableTestUtils.randomBytes(fixedValueSize); + byte[] key = HashTableTestUtils.randomBytes(2048); + ByteArrayEntry largeEntry = bigSerializer.randomEntry(key.length); - try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool, 1, -1)) { - cache.put(key, largeValue); + try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool, 1, -1)) { + cache.put(key, largeEntry); } } @@ -294,20 +280,19 @@ public void testPutTooLargeKey(HashAlgorithm hashAlgorithm, boolean useMemoryPoo @Test(dataProvider = "hashAlgorithms", dependsOnMethods = "testBasics") public void testAddOrReplace(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws Exception { - try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) + try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) { - byte[] oldValue = null; + ByteArrayEntry oldEntry = null; for (int i = 0; i < HashTableTestUtils.manyCount; i++) - assertTrue(cache.addOrReplace(Longs.toByteArray(i), oldValue, HashTableTestUtils - .randomBytes(fixedValueSize))); + assertTrue(cache.addOrReplace(Longs.toByteArray(i), oldEntry, serializer.randomEntry(8))); byte[] key = Longs.toByteArray(42); - byte[] value = cache.get(key); - byte[] update1 = HashTableTestUtils.randomBytes(fixedValueSize); - assertTrue(cache.addOrReplace(key, value, update1)); + ByteArrayEntry entry = cache.get(key); + ByteArrayEntry update1 = serializer.randomEntry(key.length); + assertTrue(cache.addOrReplace(key, entry, update1)); Assert.assertEquals(cache.get(key), update1); - byte[] update2 = HashTableTestUtils.randomBytes(fixedValueSize); + ByteArrayEntry update2 = serializer.randomEntry(key.length); assertTrue(cache.addOrReplace(key, update1, update2)); Assert.assertEquals(cache.get(key), update2); Assert.assertFalse(cache.addOrReplace(key, update1, update2)); @@ -316,7 +301,7 @@ public void testAddOrReplace(HashAlgorithm hashAlgorithm, boolean useMemoryPool) cache.remove(key); Assert.assertNull(cache.get(key)); - byte[] update3 = HashTableTestUtils.randomBytes(fixedValueSize); + ByteArrayEntry update3 = serializer.randomEntry(key.length); // update will fail since the key was removed but old value is non-null. Assert.assertFalse(cache.addOrReplace(key, update2, update3)); @@ -327,30 +312,30 @@ public void testAddOrReplace(HashAlgorithm hashAlgorithm, boolean useMemoryPool) @Test(dataProvider = "hashAlgorithms") public void testPutIfAbsent(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws Exception { - try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) + try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) { for (int i = 0; i < HashTableTestUtils.manyCount; i++) - assertTrue(cache.putIfAbsent(Longs.toByteArray(i), HashTableTestUtils.randomBytes(fixedValueSize))); + assertTrue(cache.putIfAbsent(Longs.toByteArray(i), serializer.randomEntry(8))); 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)); + ByteArrayEntry entry = serializer.randomEntry(key.length); + assertTrue(cache.putIfAbsent(key, entry)); + Assert.assertEquals(cache.get(key), entry); + Assert.assertFalse(cache.putIfAbsent(key, entry)); } } @Test(dataProvider = "hashAlgorithms") public void testRemove(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws Exception { - try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) + try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) { - HashTableTestUtils.fillMany(cache, fixedValueSize); + HashTableTestUtils.fillMany(cache, serializer); byte[] key = Longs.toByteArray(HashTableTestUtils.manyCount + 100); - byte[] value = HashTableTestUtils.randomBytes(fixedValueSize); - cache.put(key, value); - Assert.assertEquals(cache.get(key), value); + ByteArrayEntry entry = serializer.randomEntry(key.length); + cache.put(key, entry); + Assert.assertEquals(cache.get(key), entry); cache.remove(key); Assert.assertNull(cache.get(key)); Assert.assertFalse(cache.remove(key)); @@ -364,16 +349,15 @@ public void testRemove(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throw @Test(dataProvider = "hashAlgorithms") public void testClear(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws Exception { - try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) + try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) { - List data = new ArrayList<>(); + List data = new ArrayList<>(); for (int i = 0; i < 100; i++) { - data.add(new HashTableTestUtils.KeyValuePair(Longs.toByteArray(i), HashTableTestUtils - .randomBytes(fixedValueSize))); + data.add(new KeyEntryPair(Longs.toByteArray(i), serializer.randomEntry(8))); } - data.forEach(kv -> cache.put(kv.key, kv.value)); - data.forEach(kv -> Assert.assertEquals(cache.get(kv.key), kv.value)); + data.forEach(kv -> cache.put(kv.key, kv.entry)); + data.forEach(kv -> Assert.assertEquals(cache.get(kv.key), kv.entry)); assertEquals(cache.size(), 100); @@ -385,35 +369,35 @@ public void testClear(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws @Test(dataProvider = "hashAlgorithms") public void testGet_Put(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws Exception { - try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) + 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); + ByteArrayEntry entry = serializer.randomEntry(key.length); + cache.put(key, entry); + assertEquals(cache.get(key), entry); 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); + ByteArrayEntry entry11 = serializer.randomEntry(key11.length); + cache.put(key11, entry11); + Assert.assertEquals(cache.get(key), entry); + Assert.assertEquals(cache.get(key11), entry11); + + entry11 = serializer.randomEntry(key11.length); + cache.put(key11, entry11); + Assert.assertEquals(cache.get(key), entry); + Assert.assertEquals(cache.get(key11), entry11); } } @Test(dataProvider = "hashAlgorithms") public void testContainsKey(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws Exception { - try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) + try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) { byte[] key = Longs.toByteArray(42); - byte[] value = HashTableTestUtils.randomBytes(fixedValueSize); - cache.put(key, value); + ByteArrayEntry entry = serializer.randomEntry(key.length); + cache.put(key, entry); assertTrue(cache.containsKey(key)); Assert.assertFalse(cache.containsKey(Longs.toByteArray(11))); } @@ -423,15 +407,14 @@ public void testContainsKey(HashAlgorithm hashAlgorithm, boolean useMemoryPool) @Test(dataProvider = "hashAlgorithms") public void testGetBucketHistogram(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws Exception { - try (DoubleCheckOffHeapHashTableImpl cache = cache(hashAlgorithm, useMemoryPool)) + try (DoubleCheckOffHeapHashTableImpl cache = cache(hashAlgorithm, useMemoryPool)) { - List data = new ArrayList<>(); + List data = new ArrayList<>(); for (int i = 0; i < 100; i++) { - data.add(new HashTableTestUtils.KeyValuePair(Longs.toByteArray(i), HashTableTestUtils - .randomBytes(fixedValueSize))); + data.add(new KeyEntryPair(Longs.toByteArray(i), serializer.randomEntry(8))); } - data.forEach(kv -> cache.put(kv.key, kv.value)); + data.forEach(kv -> cache.put(kv.key, kv.entry)); Assert.assertEquals(cache.stats().getSize(), 100); @@ -460,13 +443,13 @@ private static int sum(int[] ints) @Test(dataProvider = "hashAlgorithms") public void testResetStatistics(HashAlgorithm hashAlgorithm, boolean useMemoryPool) throws IOException { - try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) + try (OffHeapHashTable cache = cache(hashAlgorithm, useMemoryPool)) { for (int i = 0; i < 100; i++) - cache.put(Longs.toByteArray(i), HashTableTestUtils.randomBytes(fixedValueSize)); + cache.put(Longs.toByteArray(i), serializer.randomEntry(8)); for (int i = 0; i < 30; i++) - cache.put(Longs.toByteArray(i), HashTableTestUtils.randomBytes(fixedValueSize)); + cache.put(Longs.toByteArray(i), serializer.randomEntry(8)); for (int i = 0; i < 50; i++) cache.get(Longs.toByteArray(i)); diff --git a/src/test/java/com/oath/halodb/DBRepairTest.java b/src/test/java/com/oath/halodb/DBRepairTest.java index b4cb96b..0a0bdfe 100644 --- a/src/test/java/com/oath/halodb/DBRepairTest.java +++ b/src/test/java/com/oath/halodb/DBRepairTest.java @@ -5,9 +5,6 @@ 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; @@ -16,6 +13,9 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.testng.Assert; +import org.testng.annotations.Test; + public class DBRepairTest extends TestBase { @Test(dataProvider = "Options") @@ -26,9 +26,9 @@ public void testRepairDB(HaloDBOptions options) throws HaloDBException, IOExcept options.setCompactionDisabled(true); HaloDB db = getTestDB(directory, options); - int noOfRecords = 5 * 1024 + 512; // 5 files with 1024 records and 1 with 512 records. + 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); + List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-RecordEntry.Header.HEADER_SIZE); // delete half the records. for (int i = 0; i < noOfRecords; i++) { @@ -88,7 +88,7 @@ public void testRepairDBWithCompaction(HaloDBOptions options) throws HaloDBExcep HaloDB db = getTestDB(directory, options); int noOfRecords = 10 * 1024 + 512; - List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-Record.Header.HEADER_SIZE); + List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-RecordEntry.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++) { diff --git a/src/test/java/com/oath/halodb/DataConsistencyTest.java b/src/test/java/com/oath/halodb/DataConsistencyTest.java index 321b12d..85c845d 100644 --- a/src/test/java/com/oath/halodb/DataConsistencyTest.java +++ b/src/test/java/com/oath/halodb/DataConsistencyTest.java @@ -5,7 +5,10 @@ package com.oath.halodb; -import com.google.common.primitives.Longs; +import java.nio.ByteBuffer; +import java.util.HashSet; +import java.util.Random; +import java.util.Set; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -13,10 +16,7 @@ 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; +import com.google.common.primitives.Longs; public class DataConsistencyTest extends TestBase { private static final Logger logger = LoggerFactory.getLogger(DataConsistencyTest.class); @@ -35,7 +35,7 @@ public class DataConsistencyTest extends TestBase { private ByteBuffer[] keys; private RandomDataGenerator randDataGenerator; - private Random random = new Random(); + private final Random random = new Random(); private HaloDB haloDB; diff --git a/src/test/java/com/oath/halodb/DoubleCheckOffHeapHashTableImpl.java b/src/test/java/com/oath/halodb/DoubleCheckOffHeapHashTableImpl.java index 2eec5f2..ecae117 100644 --- a/src/test/java/com/oath/halodb/DoubleCheckOffHeapHashTableImpl.java +++ b/src/test/java/com/oath/halodb/DoubleCheckOffHeapHashTableImpl.java @@ -7,46 +7,49 @@ package com.oath.halodb; -import com.google.common.primitives.Longs; -import com.oath.halodb.histo.EstimatedHistogram; +import java.io.IOException; +import java.util.Arrays; import org.testng.Assert; -import java.io.IOException; +import com.oath.halodb.histo.EstimatedHistogram; /** * 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 class DoubleCheckOffHeapHashTableImpl implements OffHeapHashTable { - public final OffHeapHashTable prod; - public final OffHeapHashTable check; + public final OffHeapHashTable prod; + public final OffHeapHashTable check; - public DoubleCheckOffHeapHashTableImpl(OffHeapHashTableBuilder builder) + public DoubleCheckOffHeapHashTableImpl(OffHeapHashTableBuilder builder) { this.prod = builder.build(); this.check = new CheckOffHeapHashTable<>(builder); } - public boolean put(byte[] key, V value) + @Override + public boolean put(byte[] key, E entry) { - boolean rProd = prod.put(key, value); - boolean rCheck = check.put(key, value); + boolean rProd = prod.put(key, entry); + boolean rCheck = check.put(key, entry); Assert.assertEquals(rProd, rCheck, "for key='" + key + '\''); return rProd; } - public boolean addOrReplace(byte[] key, V old, V value) + @Override + public boolean addOrReplace(byte[] key, E old, E entry) { - boolean rProd = prod.addOrReplace(key, old, value); - boolean rCheck = check.addOrReplace(key, old, value); + boolean rProd = prod.addOrReplace(key, old, entry); + boolean rCheck = check.addOrReplace(key, old, entry); Assert.assertEquals(rProd, rCheck, "for key='" + key + '\''); return rProd; } - public boolean putIfAbsent(byte[] k, V v) + @Override + public boolean putIfAbsent(byte[] k, E v) { boolean rProd = prod.putIfAbsent(k, v); boolean rCheck = check.putIfAbsent(k, v); @@ -54,16 +57,17 @@ public boolean putIfAbsent(byte[] k, V v) return rProd; } - public boolean putIfAbsent(byte[] key, V value, long expireAt) + public boolean putIfAbsent(byte[] key, E entry, long expireAt) { throw new UnsupportedOperationException(); } - public boolean put(byte[] key, V value, long expireAt) + public boolean put(byte[] key, E entry, long expireAt) { throw new UnsupportedOperationException(); } + @Override public boolean remove(byte[] key) { boolean rProd = prod.remove(key); @@ -72,34 +76,39 @@ public boolean remove(byte[] key) return rProd; } + @Override public void clear() { prod.clear(); check.clear(); } - public V get(byte[] key) + @Override + public E get(byte[] key) { - V rProd = prod.get(key); - V rCheck = check.get(key); - Assert.assertEquals(rProd, rCheck, "for key='" + Longs.fromByteArray(key) + '\''); + E rProd = prod.get(key); + E rCheck = check.get(key); + Assert.assertEquals(rProd, rCheck, "for key=" + Arrays.toString(key)); return rProd; } + @Override public boolean containsKey(byte[] key) { boolean rProd = prod.containsKey(key); boolean rCheck = check.containsKey(key); - Assert.assertEquals(rProd, rCheck, "for key='" + key + '\''); + Assert.assertEquals(rProd, rCheck, "for key=" + Arrays.toString(key)); return rProd; } + @Override public void resetStatistics() { prod.resetStatistics(); check.resetStatistics(); } + @Override public long size() { long rProd = prod.size(); @@ -108,6 +117,7 @@ public long size() return rProd; } + @Override public int[] hashTableSizes() { return prod.hashTableSizes(); @@ -121,11 +131,13 @@ public SegmentStats[] perSegmentStats() { return rProd; } + @Override public EstimatedHistogram getBucketHistogram() { return prod.getBucketHistogram(); } + @Override public int segments() { int rProd = prod.segments(); @@ -134,6 +146,7 @@ public int segments() return rProd; } + @Override public float loadFactor() { float rProd = prod.loadFactor(); @@ -142,6 +155,7 @@ public float loadFactor() return rProd; } + @Override public OffHeapHashTableStats stats() { OffHeapHashTableStats rProd = prod.stats(); @@ -150,6 +164,7 @@ public OffHeapHashTableStats stats() return rProd; } + @Override public void close() throws IOException { prod.close(); diff --git a/src/test/java/com/oath/halodb/HaloDBCompactionTest.java b/src/test/java/com/oath/halodb/HaloDBCompactionTest.java index cee8325..abd8392 100644 --- a/src/test/java/com/oath/halodb/HaloDBCompactionTest.java +++ b/src/test/java/com/oath/halodb/HaloDBCompactionTest.java @@ -5,17 +5,17 @@ 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; +import org.testng.Assert; +import org.testng.annotations.Test; + +import com.google.common.primitives.Longs; + public class HaloDBCompactionTest extends TestBase { private final int recordSize = 1024; @@ -142,17 +142,17 @@ public void testFilesWithStaleDataAddedToCompactionQueueDuringDBOpen(HaloDBOptio HaloDB db = getTestDB(directory, options); // insert 50 records into 5 files. - List records = TestUtils.insertRandomRecordsOfSize(db, 50, 1024-Record.Header.HEADER_SIZE); + List records = TestUtils.insertRandomRecordsOfSize(db, 50, 1024-RecordEntry.Header.HEADER_SIZE); // Delete all records, which means that all data files would have crossed the - // stale data threshold. + // stale data threshold. for (Record r : records) { db.delete(r.getKey()); } db.close(); - // open the db withe compaction enabled. + // open the db withe compaction enabled. options.setCompactionDisabled(false); options.setMaxFileSize(10 * 1024); @@ -172,8 +172,8 @@ public void testFilesWithStaleDataAddedToCompactionQueueDuringDBOpen(HaloDBOptio db = getTestDBWithoutDeletingFiles(directory, options); - // insert 20 records into two files. - records = TestUtils.insertRandomRecordsOfSize(db, 20, 1024-Record.Header.HEADER_SIZE); + // insert 20 records into two files. + records = TestUtils.insertRandomRecordsOfSize(db, 20, 1024-RecordEntry.Header.HEADER_SIZE); File[] dataFilesToDelete = FileUtils.listDataFiles(new File(directory)); // update all records; since compaction is disabled no file is deleted. @@ -212,14 +212,14 @@ public void testPauseAndResumeCompaction() throws HaloDBException, InterruptedEx // insert 100 records of size 1kb into 100 files. int noOfRecords = 1000; - List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024 - Record.Header.HEADER_SIZE); + List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024 - RecordEntry.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.updateRecordsWithSize(db, recordsToUpdate, 1024-RecordEntry.Header.HEADER_SIZE); TestUtils.waitForCompactionToComplete(db); // compaction was paused, therefore no compaction files must be present. @@ -262,7 +262,7 @@ public void testPauseAndResumeCompaction() throws HaloDBException, InterruptedEx } private Record[] insertAndUpdateRecords(int numberOfRecords, HaloDB db) throws HaloDBException { - int valueSize = recordSize - Record.Header.HEADER_SIZE - 8; // 8 is the key size. + int valueSize = recordSize - RecordEntry.Header.HEADER_SIZE - 8; // 8 is the key size. Record[] records = new Record[numberOfRecords]; for (int i = 0; i < numberOfRecords; i++) { @@ -288,7 +288,7 @@ private Record[] insertAndUpdateRecords(int numberOfRecords, HaloDB db) throws H } private Record[] insertAndUpdateRecordsToSameFile(int numberOfRecords, HaloDB db) throws HaloDBException { - int valueSize = recordSize - Record.Header.HEADER_SIZE - 8; // 8 is the key size. + int valueSize = recordSize - RecordEntry.Header.HEADER_SIZE - 8; // 8 is the key size. Record[] records = new Record[numberOfRecords]; for (int i = 0; i < numberOfRecords; i++) { diff --git a/src/test/java/com/oath/halodb/HaloDBDeletionTest.java b/src/test/java/com/oath/halodb/HaloDBDeletionTest.java index 725ab35..a3fd352 100644 --- a/src/test/java/com/oath/halodb/HaloDBDeletionTest.java +++ b/src/test/java/com/oath/halodb/HaloDBDeletionTest.java @@ -5,9 +5,6 @@ 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; @@ -15,6 +12,9 @@ import java.util.Random; import java.util.Set; +import org.testng.Assert; +import org.testng.annotations.Test; + public class HaloDBDeletionTest extends TestBase { @Test(dataProvider = "Options") @@ -222,7 +222,7 @@ public void testDeleteAllRecords(HaloDBOptions options) throws Exception { int noOfRecords = 10_000; // There will be 1000 files each of size 10KB - List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024 - Record.Header.HEADER_SIZE); + List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024 - RecordEntry.Header.HEADER_SIZE); // delete all records. for (Record r : records) { @@ -238,7 +238,7 @@ public void testDeleteAllRecords(HaloDBOptions options) throws Exception { } // only the current write file will be remaining everything else should have been - // deleted by the compaction job. + // deleted by the compaction job. Assert.assertEquals(FileUtils.listDataFiles(new File(directory)).length, 1); } } diff --git a/src/test/java/com/oath/halodb/HaloDBFileCompactionTest.java b/src/test/java/com/oath/halodb/HaloDBFileCompactionTest.java index 5fd9635..648921b 100644 --- a/src/test/java/com/oath/halodb/HaloDBFileCompactionTest.java +++ b/src/test/java/com/oath/halodb/HaloDBFileCompactionTest.java @@ -5,18 +5,18 @@ 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; +import org.hamcrest.MatcherAssert; +import org.hamcrest.Matchers; +import org.testng.annotations.Test; + +import com.google.common.primitives.Longs; + public class HaloDBFileCompactionTest extends TestBase { @Test(dataProvider = "Options") @@ -31,7 +31,7 @@ public void testCompaction(HaloDBOptions options) throws Exception { HaloDB db = getTestDB(directory, options); - byte[] data = new byte[recordSize - Record.Header.HEADER_SIZE - 8 - 8]; + byte[] data = new byte[recordSize - RecordEntry.Header.HEADER_SIZE - 8 - 8]; for (int i = 0; i < data.length; i++) { data[i] = (byte)i; } @@ -46,7 +46,7 @@ public void testCompaction(HaloDBOptions options) throws Exception { List freshRecords = new ArrayList<>(); - // There are two data files. make the first half of both the files stale. + // 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()); diff --git a/src/test/java/com/oath/halodb/HaloDBFileTest.java b/src/test/java/com/oath/halodb/HaloDBFileTest.java index 7959a26..ac7ee15 100644 --- a/src/test/java/com/oath/halodb/HaloDBFileTest.java +++ b/src/test/java/com/oath/halodb/HaloDBFileTest.java @@ -5,11 +5,6 @@ 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; @@ -17,16 +12,22 @@ import java.nio.file.Paths; import java.nio.file.StandardOpenOption; import java.nio.file.attribute.FileTime; +import java.util.ArrayList; import java.util.List; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + public class HaloDBFileTest { - private File directory = Paths.get("tmp", "HaloDBFileTest", "testIndexFile").toFile(); + private final 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 final int fileId = 100; + private final File backingFile = directory.toPath().resolve(fileId+HaloDBFile.DATA_FILE_NAME).toFile(); private FileTime createdTime; @BeforeMethod @@ -55,15 +56,17 @@ public void after() throws IOException { @Test public void testIndexFile() throws IOException { - List list = insertTestRecords(); + List list = TestUtils.generateRandomData(100); + List metas = insertTestRecords(list); indexFile.open(); - verifyIndexFile(indexFile, list); + verifyIndexFile(indexFile, list, metas); } @Test public void testFileWithInvalidRecord() throws IOException { - List list = insertTestRecords(); + List list = TestUtils.generateRandomData(100); + insertTestRecords(list); // write a corrupted header to file. try(FileChannel channel = FileChannel.open(Paths.get(directory.getCanonicalPath(), fileId + HaloDBFile.DATA_FILE_NAME).toAbsolutePath(), StandardOpenOption.APPEND)) { @@ -80,23 +83,27 @@ public void testFileWithInvalidRecord() throws IOException { // 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. + // 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(); + List list = TestUtils.generateRandomData(100); + insertTestRecords(list); // 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)); + // value length is corrupted. + RecordEntry notYetCorrupted = RecordEntry.newEntry(key, value, 1234); try(FileChannel channel = FileChannel.open(Paths.get(directory.getCanonicalPath(), fileId + HaloDBFile.DATA_FILE_NAME).toAbsolutePath(), StandardOpenOption.APPEND)) { - channel.write(corrupted.serialize()); + ByteBuffer[] data = notYetCorrupted.serialize(); + data[0].putInt(RecordEntry.Header.VALUE_SIZE_OFFSET, -3333); + int newChecksum = Utils.toSignedIntFromLong(notYetCorrupted.computeCheckSum(data[0].array())); + data[0].putInt(RecordEntry.Header.CHECKSUM_OFFSET, newChecksum); + channel.write(data); } HaloDBFile.HaloDBFileIterator iterator = file.newIterator(); @@ -116,27 +123,28 @@ public void testCorruptedHeader() throws IOException { @Test public void testRebuildIndexFile() throws IOException { - List list = insertTestRecords(); + List list = TestUtils.generateRandomData(100); + List metas = insertTestRecords(list); indexFile.delete(); - // make sure that the file is deleted. + // 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); + verifyIndexFile(indexFile, list, metas); } @Test public void testRepairDataFileWithCorruptedValue() throws IOException { - List list = insertTestRecords(); + List list = TestUtils.generateRandomData(100); + List metas = insertTestRecords(list); // 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. + // 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)); + RecordEntry record = RecordEntry.newEntry(key, value, 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()); @@ -147,19 +155,19 @@ public void testRepairDataFileWithCorruptedValue() throws IOException { Assert.assertNotEquals(TestUtils.getFileCreationTime(backingFile), createdTime); Assert.assertEquals(repairedFile.getPath(), file.getPath()); verifyDataFile(list, repairedFile); - verifyIndexFile(repairedFile.getIndexFile(), list); + verifyIndexFile(repairedFile.getIndexFile(), list, metas); } @Test public void testRepairDataFileWithInCompleteRecord() throws IOException { - List list = insertTestRecords(); + List list = TestUtils.generateRandomData(100); + List metas = insertTestRecords(list); // write a corrupted record to file. - // value was not completely written 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)); + RecordEntry record = RecordEntry.newEntry(key, value, 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()); @@ -170,12 +178,13 @@ public void testRepairDataFileWithInCompleteRecord() throws IOException { Assert.assertNotEquals(TestUtils.getFileCreationTime(backingFile), createdTime); Assert.assertEquals(repairedFile.getPath(), file.getPath()); verifyDataFile(list, repairedFile); - verifyIndexFile(repairedFile.getIndexFile(), list); + verifyIndexFile(repairedFile.getIndexFile(), list, metas); } @Test public void testRepairDataFileContainingRecordsWithCorruptedHeader() throws IOException { - List list = insertTestRecords(); + List list = TestUtils.generateRandomData(100); + List metas = insertTestRecords(list); // write a corrupted header to file. try(FileChannel channel = FileChannel.open(Paths.get(directory.getCanonicalPath(), fileId + HaloDBFile.DATA_FILE_NAME).toAbsolutePath(), StandardOpenOption.APPEND)) { @@ -187,21 +196,24 @@ public void testRepairDataFileContainingRecordsWithCorruptedHeader() throws IOEx Assert.assertNotEquals(TestUtils.getFileCreationTime(backingFile), createdTime); Assert.assertEquals(repairedFile.getPath(), file.getPath()); verifyDataFile(list, repairedFile); - verifyIndexFile(repairedFile.getIndexFile(), list); + verifyIndexFile(repairedFile.getIndexFile(), list, metas); } @Test public void testRepairDataFileContainingRecordsWithValidButCorruptedHeader() throws IOException { - List list = insertTestRecords(); + List list = TestUtils.generateRandomData(100); + List metas = insertTestRecords(list); // 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)); + // header is valid but the value size is incorrect. + RecordEntry notYetCorrupted = RecordEntry.newEntry(key, value, 1234); try(FileChannel channel = FileChannel.open(Paths.get(directory.getCanonicalPath(), fileId + HaloDBFile.DATA_FILE_NAME).toAbsolutePath(), StandardOpenOption.APPEND)) { - ByteBuffer[] data = record.serialize(); + ByteBuffer[] data = notYetCorrupted.serialize(); + data[0].putInt(RecordEntry.Header.VALUE_SIZE_OFFSET, 5); + int newChecksum = Utils.toSignedIntFromLong(notYetCorrupted.computeCheckSum(data[0].array())); + data[0].putInt(RecordEntry.Header.CHECKSUM_OFFSET, newChecksum); channel.write(data); } @@ -209,33 +221,33 @@ public void testRepairDataFileContainingRecordsWithValidButCorruptedHeader() thr Assert.assertNotEquals(TestUtils.getFileCreationTime(backingFile), createdTime); Assert.assertEquals(repairedFile.getPath(), file.getPath()); verifyDataFile(list, repairedFile); - verifyIndexFile(repairedFile.getIndexFile(), list); + verifyIndexFile(repairedFile.getIndexFile(), list, metas); } - private void verifyIndexFile(IndexFile file, List recordList) throws IOException { + private void verifyIndexFile(IndexFile file, List recordList, List metas) 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(); + Record r = recordList.get(count); + InMemoryIndexMetaData m = metas.get(count++); Assert.assertEquals(e.getKey(), r.getKey()); - int expectedOffset = meta.getValueOffset() - Record.Header.HEADER_SIZE - r.getKey().length; + int expectedOffset = m.getValueOffset() - RecordEntry.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); + private List insertTestRecords(List list) throws IOException { + List result = new ArrayList<>(); for (Record record : list) { - record.setSequenceNumber(100); - InMemoryIndexMetaData meta = file.writeRecord(record); - record.setRecordMetaData(meta); + RecordEntry entry = RecordEntry.newEntry(record, 100); + InMemoryIndexMetaData indexMeta = file.writeRecord(entry); + result.add(indexMeta); } - return list; + return result; } private void verifyDataFile(List recordList, HaloDBFile dataFile) throws IOException { @@ -246,7 +258,6 @@ private void verifyDataFile(List recordList, HaloDBFile dataFile) throws Record expected = recordList.get(count++); Assert.assertEquals(actual, expected); } - Assert.assertEquals(count, recordList.size()); } } diff --git a/src/test/java/com/oath/halodb/HaloDBIteratorTest.java b/src/test/java/com/oath/halodb/HaloDBIteratorTest.java index 9f39f40..48fe2e8 100644 --- a/src/test/java/com/oath/halodb/HaloDBIteratorTest.java +++ b/src/test/java/com/oath/halodb/HaloDBIteratorTest.java @@ -5,17 +5,17 @@ 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 org.hamcrest.MatcherAssert; +import org.hamcrest.Matchers; +import org.testng.Assert; +import org.testng.annotations.Test; + import mockit.Invocation; import mockit.Mock; import mockit.MockUp; @@ -50,7 +50,7 @@ public void testWithDelete(HaloDBOptions options) throws HaloDBException { HaloDBIterator iterator = db.newIterator(); Assert.assertFalse(iterator.hasNext()); - // close and open the db again. + // close and open the db again. db.close(); db = getTestDBWithoutDeletingFiles(directory, options); iterator = db.newIterator(); @@ -118,7 +118,7 @@ public void testPutUpdateCompactAndGetDB(HaloDBOptions options) throws HaloDBExc } // Test to make sure that no exceptions are thrown when files are being deleted by - // compaction thread and db is being iterated. + // compaction thread and db is being iterated. @Test(dataProvider = "Options") public void testConcurrentCompactionAndIterator(HaloDBOptions options) throws HaloDBException, InterruptedException { String directory = TestUtils.getTestDirectory("HaloDBIteratorTest", "testConcurrentCompactionAndIterator"); @@ -130,7 +130,7 @@ public void testConcurrentCompactionAndIterator(HaloDBOptions options) throws Ha // 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); + List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-RecordEntry.Header.HEADER_SIZE); int noOfUpdateRuns = 10; Thread updateThread = new Thread(() -> { @@ -138,7 +138,7 @@ public void testConcurrentCompactionAndIterator(HaloDBOptions options) throws Ha TestUtils.updateRecordsWithSize(db, records, 1024); } }); - // start updating the records. + // start updating the records. updateThread.start(); while (updateThread.isAlive()) { @@ -174,8 +174,8 @@ byte[] readFromFile(Invocation invocation, int offset, int length) throws HaloDB 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 noOfRecords = 4; // 2 records on 2 files. + List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-RecordEntry.Header.HEADER_SIZE); int noOfUpdateRuns = 1000; Thread updateThread = new Thread(() -> { @@ -197,7 +197,7 @@ byte[] readFromFile(Invocation invocation, int offset, int length) throws HaloDB @Test(dataProvider = "Options") public void testConcurrentCompactionAndIteratorWithMockedException(HaloDBOptions options) throws HaloDBException { // Previous tests are not guaranteed to throw ClosedChannelException. Here we throw a mock exception - // to make sure that iterator gracefully handles files being closed and delete by compaction thread. + // to make sure that iterator gracefully handles files being closed and delete by compaction thread. String directory = TestUtils.getTestDirectory("HaloDBIteratorTest", "testConcurrentCompactionAndIteratorWithMockedException"); @@ -219,7 +219,7 @@ byte[] readFromFile(Invocation invocation, int offset, int length) throws IOExce 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); + List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-RecordEntry.Header.HEADER_SIZE); int noOfUpdateRuns = 100; Thread updateThread = new Thread(() -> { diff --git a/src/test/java/com/oath/halodb/HaloDBStatsTest.java b/src/test/java/com/oath/halodb/HaloDBStatsTest.java index 407ef3e..37aa2d9 100644 --- a/src/test/java/com/oath/halodb/HaloDBStatsTest.java +++ b/src/test/java/com/oath/halodb/HaloDBStatsTest.java @@ -5,13 +5,13 @@ 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; +import org.testng.Assert; +import org.testng.annotations.Test; + public class HaloDBStatsTest extends TestBase { @Test(dataProvider = "Options") @@ -50,11 +50,11 @@ public void testStaleMap(HaloDBOptions options) throws HaloDBException { HaloDB db = getTestDB(dir, options); - // will create 10 files with 10 records each. - int recordSize = 1024 - Record.Header.HEADER_SIZE; + // will create 10 files with 10 records each. + int recordSize = 1024 - RecordEntry.Header.HEADER_SIZE; List records = TestUtils.insertRandomRecordsOfSize(db, 100, recordSize); - // No updates hence stale data map should be empty. + // No updates hence stale data map should be empty. Assert.assertEquals(db.stats().getStaleDataPercentPerFile().size(), 0); for (int i = 0; i < records.size(); i++) { @@ -62,7 +62,7 @@ public void testStaleMap(HaloDBOptions options) throws HaloDBException { db.put(records.get(i).getKey(), TestUtils.generateRandomByteArray(recordSize)); } - // Updated 1 out of 10 records in each file, hence 10% stale data. + // 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); @@ -82,14 +82,14 @@ public void testCompactionStats(HaloDBOptions options) throws HaloDBException { HaloDB db = getTestDB(dir, options); // will create 10 files with 10 records each. - int recordSize = 1024 - Record.Header.HEADER_SIZE; + int recordSize = 1024 - RecordEntry.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. + // 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)); @@ -113,7 +113,7 @@ public void testCompactionStats(HaloDBOptions options) throws HaloDBException { TestUtils.waitForCompactionToComplete(db); - // compaction complete hence stale data map is empty. + // compaction complete hence stale data map is empty. HaloDBStats stats = db.stats(); Assert.assertEquals(stats.getStaleDataPercentPerFile().size(), 0); diff --git a/src/test/java/com/oath/halodb/HashEntrySerializerTest.java b/src/test/java/com/oath/halodb/HashEntrySerializerTest.java new file mode 100644 index 0000000..11ae368 --- /dev/null +++ b/src/test/java/com/oath/halodb/HashEntrySerializerTest.java @@ -0,0 +1,38 @@ +/* + * 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.function.BiPredicate; + +import org.testng.Assert; + +interface HashEntrySerializerTest { + + default E testSerDe(E entry, HashEntrySerializer serializer, BiPredicate equals) { + long adr = Uns.allocate(serializer.entrySize(), true); + try { + entry.serializeSizes(adr); + Assert.assertTrue(entry.compareSizes(adr)); + + long locationAdr = adr + serializer.sizesSize(); + entry.serializeLocation(locationAdr); + Assert.assertTrue(entry.compareLocation(locationAdr)); + + Assert.assertTrue(entry.compare(adr, locationAdr)); + + Assert.assertEquals(serializer.readKeySize(adr), entry.getKeySize()); + + E fromAdr = serializer.deserialize(adr, locationAdr); + + Assert.assertEquals(fromAdr.getKeySize(), entry.getKeySize()); + Assert.assertEquals(fromAdr.getValueSize(), entry.getValueSize()); + Assert.assertTrue(equals.test(fromAdr, entry)); + return fromAdr; + } finally { + Uns.free(adr); + } + } +} diff --git a/src/test/java/com/oath/halodb/HashTableTestUtils.java b/src/test/java/com/oath/halodb/HashTableTestUtils.java index 0c79b2a..51449e1 100644 --- a/src/test/java/com/oath/halodb/HashTableTestUtils.java +++ b/src/test/java/com/oath/halodb/HashTableTestUtils.java @@ -7,269 +7,41 @@ 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; - } - }; +final class HashTableTestUtils { + static int manyCount = 20000; - static int writeUTFLen(String str) + static List fillMany(OffHeapHashTable cache, ByteArrayEntrySerializer serializer) { - 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; + return fill(cache, serializer, manyCount); } - 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 intSerializerFailSerialize = new HashTableValueSerializer() + static List fill(OffHeapHashTable cache, ByteArrayEntrySerializer serializer, int count) { - public void serialize(Integer s, ByteBuffer buf) - { - throw new RuntimeException("foo bar"); - } - - 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 intSerializerFailDeserialize = new HashTableValueSerializer() - { - public void serialize(Integer s, ByteBuffer buf) - { - buf.putInt(s); - } - - public Integer deserialize(ByteBuffer buf) - { - throw new RuntimeException("foo bar"); - } - - public int serializedSize(Integer s) - { - return 4; + List many = new ArrayList<>(); + for (int i = 0; i < count; i++) { + byte[] key = HashTableTestUtils.randomBytesOfRange(6, 8, 40); + ByteArrayEntry entry = serializer.randomEntry(key.length); + cache.put(key, entry); + many.add(new KeyEntryPair(key, entry)); } - }; - static final String big; - static final String bigRandom; - static { - dummyByteArray = new byte[500]; - for (int i = 0; i < HashTableTestUtils.dummyByteArray.length; i++) - HashTableTestUtils.dummyByteArray[i] = (byte) ((byte) i % 199); + return many; } - static int manyCount = 20000; - - static - { - - StringBuilder sb = new StringBuilder(); - for (int i = 0; i < 1000; i++) - sb.append("the quick brown fox jumps over the lazy dog"); - big = sb.toString(); - + /** return a byte[] that has a 50% chance of being in [min, pivot] and a 50% chance of being in (pivot, max] **/ + static byte[] randomBytesOfRange(int min, int pivot, int max) { Random r = new Random(); - sb.setLength(0); - for (int i = 0; i < 30000; i++) - sb.append((char) (r.nextInt(99) + 31)); - bigRandom = sb.toString(); - } - - static List fillMany(OffHeapHashTable cache, int fixedValueSize) - { - return fill(cache, fixedValueSize, manyCount); - } - - static List fill(OffHeapHashTable cache, int fixedValueSize, int count) - { - List many = new ArrayList<>(); - for (int i = 0; i < count; i++) { - byte[] key = Longs.toByteArray(i); - byte[] value = HashTableTestUtils.randomBytes(fixedValueSize); - cache.put(key, value); - many.add(new KeyValuePair(key, value)); + int size; + if (r.nextBoolean()) { + size = min + r.nextInt(pivot - min + 1); + } else { + size = pivot + 1 + r.nextInt(max - pivot); } - - return many; + return randomBytes(size); } static byte[] randomBytes(int len) @@ -280,12 +52,13 @@ static byte[] randomBytes(int len) return arr; } - static class KeyValuePair { - byte[] key, value; + static class KeyEntryPair { + byte[] key; + ByteArrayEntry entry; - KeyValuePair(byte[] key, byte[] value) { + KeyEntryPair(byte[] key, ByteArrayEntry entry) { this.key = key; - this.value = value; + this.entry = entry; } } } diff --git a/src/test/java/com/oath/halodb/HashTableUtilTest.java b/src/test/java/com/oath/halodb/HashTableUtilTest.java index 2e1a80f..3ed4bd0 100644 --- a/src/test/java/com/oath/halodb/HashTableUtilTest.java +++ b/src/test/java/com/oath/halodb/HashTableUtilTest.java @@ -7,8 +7,6 @@ package com.oath.halodb; -import com.oath.halodb.HashTableUtil; - import org.testng.Assert; import org.testng.annotations.Test; @@ -17,19 +15,46 @@ public class HashTableUtilTest static final long BIG = 2L << 40; @Test - public void testBitNum() - { + public void testBitNum() { Assert.assertEquals(HashTableUtil.bitNum(0), 0); Assert.assertEquals(HashTableUtil.bitNum(1), 1); Assert.assertEquals(HashTableUtil.bitNum(2), 2); + Assert.assertEquals(HashTableUtil.bitNum(3), 2); Assert.assertEquals(HashTableUtil.bitNum(4), 3); + Assert.assertEquals(HashTableUtil.bitNum(7), 3); Assert.assertEquals(HashTableUtil.bitNum(8), 4); + Assert.assertEquals(HashTableUtil.bitNum(9), 4); Assert.assertEquals(HashTableUtil.bitNum(16), 5); + Assert.assertEquals(HashTableUtil.bitNum(31), 5); Assert.assertEquals(HashTableUtil.bitNum(32), 6); + Assert.assertEquals(HashTableUtil.bitNum(33), 6); Assert.assertEquals(HashTableUtil.bitNum(64), 7); + Assert.assertEquals(HashTableUtil.bitNum(127), 7); Assert.assertEquals(HashTableUtil.bitNum(128), 8); + Assert.assertEquals(HashTableUtil.bitNum(129), 8); Assert.assertEquals(HashTableUtil.bitNum(256), 9); + Assert.assertEquals(HashTableUtil.bitNum(1023), 10); Assert.assertEquals(HashTableUtil.bitNum(1024), 11); + Assert.assertEquals(HashTableUtil.bitNum(1025), 11); + Assert.assertEquals(HashTableUtil.bitNum(65535), 16); Assert.assertEquals(HashTableUtil.bitNum(65536), 17); + Assert.assertEquals(HashTableUtil.bitNum(65537), 17); + } + + @Test + public void testRoundUpToPowerOf2() { + Assert.assertEquals(HashTableUtil.roundUpToPowerOf2(1, 6), 1); + Assert.assertEquals(HashTableUtil.roundUpToPowerOf2(2, 6), 2); + Assert.assertEquals(HashTableUtil.roundUpToPowerOf2(3, 6), 4); + Assert.assertEquals(HashTableUtil.roundUpToPowerOf2(4, 6), 4); + Assert.assertEquals(HashTableUtil.roundUpToPowerOf2(5, 6), 8); + Assert.assertEquals(HashTableUtil.roundUpToPowerOf2(8, 6), 8); + Assert.assertEquals(HashTableUtil.roundUpToPowerOf2(15, 6), 16); + Assert.assertEquals(HashTableUtil.roundUpToPowerOf2(16, 6), 16); + Assert.assertEquals(HashTableUtil.roundUpToPowerOf2(17, 6), 32); + Assert.assertEquals(HashTableUtil.roundUpToPowerOf2(32, 6), 32); + Assert.assertEquals(HashTableUtil.roundUpToPowerOf2(63, 6), 64); + Assert.assertEquals(HashTableUtil.roundUpToPowerOf2(64, 6), 64); + Assert.assertEquals(HashTableUtil.roundUpToPowerOf2(65, 6), 64); } } diff --git a/src/test/java/com/oath/halodb/HashTableValueSerializerTest.java b/src/test/java/com/oath/halodb/HashTableValueSerializerTest.java index ee27a2d..5f67625 100644 --- a/src/test/java/com/oath/halodb/HashTableValueSerializerTest.java +++ b/src/test/java/com/oath/halodb/HashTableValueSerializerTest.java @@ -7,14 +7,14 @@ package com.oath.halodb; -import com.google.common.primitives.Ints; -import com.google.common.primitives.Longs; +import java.io.IOException; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.Test; -import java.io.IOException; +import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; public class HashTableValueSerializerTest { @@ -27,14 +27,14 @@ public void deinit() @Test public void testFailingValueSerializerOnPut() throws IOException, InterruptedException { - try (OffHeapHashTable cache = OffHeapHashTableBuilder.newBuilder() - .valueSerializer(HashTableTestUtils.byteArraySerializerFailSerialize) - .fixedValueSize(8) - .build()) + ByteArrayEntrySerializer serializer = ByteArrayEntrySerializer.ofSize(8); + try (OffHeapHashTable cache = OffHeapHashTableBuilder.newBuilder(serializer).build()) { + byte[] key = Ints.toByteArray(1); + ByteArrayEntry entry = new ByteArrayEntry(key.length, Longs.toByteArray(1), true); try { - cache.put(Ints.toByteArray(1), Longs.toByteArray(1)); + cache.put(key, entry); Assert.fail(); } catch (RuntimeException ignored) @@ -44,7 +44,7 @@ public void testFailingValueSerializerOnPut() throws IOException, InterruptedExc try { - cache.putIfAbsent(Ints.toByteArray(1), Longs.toByteArray(1)); + cache.putIfAbsent(key, entry); Assert.fail(); } catch (RuntimeException ignored) @@ -54,7 +54,7 @@ public void testFailingValueSerializerOnPut() throws IOException, InterruptedExc try { - cache.addOrReplace(Ints.toByteArray(1), Longs.toByteArray(1), Longs.toByteArray(2)); + cache.addOrReplace(key, entry, entry); Assert.fail(); } catch (RuntimeException ignored) diff --git a/src/test/java/com/oath/halodb/InMemoryIndexMetaDataSerializerTest.java b/src/test/java/com/oath/halodb/InMemoryIndexMetaDataSerializerTest.java new file mode 100644 index 0000000..cd9dd09 --- /dev/null +++ b/src/test/java/com/oath/halodb/InMemoryIndexMetaDataSerializerTest.java @@ -0,0 +1,76 @@ +/* + * 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; + +public class InMemoryIndexMetaDataSerializerTest implements HashEntrySerializerTest { + + InMemoryIndexMetaDataSerializer serializer = new InMemoryIndexMetaDataSerializer(); + InMemoryIndexMetaData data1 = new InMemoryIndexMetaData(1000, 2000, 3000, 4000, 1234); + InMemoryIndexMetaData data2 = new InMemoryIndexMetaData(1, 2, 3, 4, 5); + + boolean equalData(InMemoryIndexMetaData e1, InMemoryIndexMetaData e2) { + return e1.getFileId() == e2.getFileId() + && e1.getKeySize() == e2.getKeySize() + && e1.getSequenceNumber() == e2.getSequenceNumber() + && e1.getValueOffset() == e2.getValueOffset() + && e1.getValueSize() == e2.getValueSize(); + } + + boolean nothingEqual(InMemoryIndexMetaData e1, InMemoryIndexMetaData e2) { + return e1.getFileId() != e2.getFileId() + && e1.getKeySize() != e2.getKeySize() + && e1.getSequenceNumber() != e2.getSequenceNumber() + && e1.getValueOffset() != e2.getValueOffset() + && e1.getValueSize() != e2.getValueSize(); + } + + @Test + public void testSerializeDeserialize() { + + InMemoryIndexMetaData data1read = testSerDe(data1, serializer, this::equalData); + InMemoryIndexMetaData data2read = testSerDe(data2, serializer, this::equalData); + + Assert.assertTrue(nothingEqual(data1, data2)); + Assert.assertTrue(nothingEqual(data1read, data2read)); + } + + @Test + public void testFromRecordHeader() { + RecordEntry.Header header = new RecordEntry.Header(1, (byte)2, 1000, 2000, 9999); + InMemoryIndexMetaData data = new InMemoryIndexMetaData(header, 33, 55); + Assert.assertEquals(data.getFileId(), 33); + Assert.assertEquals(data.getKeySize(), header.getKeySize()); + Assert.assertEquals(data.getValueOffset(), RecordEntry.getValueOffset(55, data.getKeySize())); + Assert.assertEquals(data.getValueSize(), header.getValueSize()); + Assert.assertEquals(data.getSequenceNumber(), header.getSequenceNumber()); + } + + @Test + public void testFromIndexEntry() { + IndexFileEntry entry = new IndexFileEntry(new byte[33], 88, 101, 5555, (byte)2, 2342323422L); + InMemoryIndexMetaData data = new InMemoryIndexMetaData(entry, 33); + Assert.assertEquals(data.getFileId(), 33); + Assert.assertEquals(data.getKeySize(), entry.getKey().length); + Assert.assertEquals(data.getValueOffset(), RecordEntry.getValueOffset(entry.getRecordOffset(), data.getKeySize())); + Assert.assertEquals(data.getValueSize(), RecordEntry.getValueSize(entry.getRecordSize(), data.getKeySize())); + Assert.assertEquals(data.getSequenceNumber(), entry.getSequenceNumber()); + } + + @Test + public void testRelocate() { + Assert.assertEquals(data1.getFileId(), 1000); + Assert.assertEquals(data1.getValueOffset(), 2000); + InMemoryIndexMetaData relocated = data1.relocated(77, 1234); + Assert.assertEquals(relocated.getFileId(), 77); + Assert.assertEquals(relocated.getKeySize(), data1.getKeySize()); + Assert.assertEquals(relocated.getValueOffset(), RecordEntry.getValueOffset(1234, data1.getKeySize())); + Assert.assertEquals(relocated.getValueSize(), data1.getValueSize()); + Assert.assertEquals(relocated.getSequenceNumber(), data1.getSequenceNumber()); + } +} diff --git a/src/test/java/com/oath/halodb/IndexFileEntryTest.java b/src/test/java/com/oath/halodb/IndexFileEntryTest.java index 5c41750..532926a 100644 --- a/src/test/java/com/oath/halodb/IndexFileEntryTest.java +++ b/src/test/java/com/oath/halodb/IndexFileEntryTest.java @@ -5,28 +5,35 @@ package com.oath.halodb; -import org.testng.Assert; -import org.testng.annotations.Test; +import static com.oath.halodb.IndexFileEntry.CHECKSUM_OFFSET; +import static com.oath.halodb.IndexFileEntry.CHECKSUM_SIZE; +import static com.oath.halodb.IndexFileEntry.INDEX_FILE_HEADER_SIZE; +import static com.oath.halodb.IndexFileEntry.KEY_SIZE_OFFSET; +import static com.oath.halodb.IndexFileEntry.RECORD_OFFSET; +import static com.oath.halodb.IndexFileEntry.RECORD_SIZE_OFFSET; +import static com.oath.halodb.IndexFileEntry.SEQUENCE_NUMBER_OFFSET; +import static com.oath.halodb.IndexFileEntry.VERSION_OFFSET; import java.nio.ByteBuffer; import java.util.zip.CRC32; -import static com.oath.halodb.IndexFileEntry.*; +import org.testng.Assert; +import org.testng.annotations.Test; public class IndexFileEntryTest { @Test public void serializeIndexFileEntry() { - byte[] key = TestUtils.generateRandomByteArray(8); + byte[] key = TestUtils.generateRandomByteArray(1234); int recordSize = 1024; int recordOffset = 10240; - byte keySize = (byte) key.length; + int keySize = key.length; long sequenceNumber = 100; - int version = 200; + byte version = 20; ByteBuffer header = ByteBuffer.allocate(INDEX_FILE_HEADER_SIZE); - header.put(VERSION_OFFSET, (byte)version); - header.put(KEY_SIZE_OFFSET, keySize); + header.put(VERSION_OFFSET, Utils.versionByte(version, keySize)); + header.put(KEY_SIZE_OFFSET, Utils.keySizeByte(keySize)); header.putInt(RECORD_SIZE_OFFSET, recordSize); header.putInt(RECORD_OFFSET, recordOffset); header.putLong(SEQUENCE_NUMBER_OFFSET, sequenceNumber); @@ -46,18 +53,18 @@ public void serializeIndexFileEntry() { @Test public void deserializeIndexFileEntry() { - byte[] key = TestUtils.generateRandomByteArray(8); + byte[] key = TestUtils.generateRandomByteArray(300); int recordSize = 1024; int recordOffset = 10240; - byte keySize = (byte) key.length; + short keySize = (short) key.length; long sequenceNumber = 100; int version = 10; long checksum = 42323; ByteBuffer header = ByteBuffer.allocate(IndexFileEntry.INDEX_FILE_HEADER_SIZE + keySize); header.putInt((int)checksum); - header.put((byte)version); - header.put(keySize); + header.put((byte)((version << 3) | (keySize >>> 8))); + header.put((byte)(keySize & 0xFF)); header.putInt(recordSize); header.putInt(recordOffset); header.putLong(sequenceNumber); diff --git a/src/test/java/com/oath/halodb/KeyBufferTest.java b/src/test/java/com/oath/halodb/KeyBufferTest.java index 4d63843..ca51477 100644 --- a/src/test/java/com/oath/halodb/KeyBufferTest.java +++ b/src/test/java/com/oath/halodb/KeyBufferTest.java @@ -7,19 +7,19 @@ package com.oath.halodb; -import com.google.common.hash.HashCode; -import com.google.common.hash.Hasher; -import com.google.common.hash.Hashing; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + +import java.nio.ByteBuffer; import org.testng.annotations.AfterMethod; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.nio.ByteBuffer; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertTrue; +import com.google.common.hash.HashCode; +import com.google.common.hash.Hasher; +import com.google.common.hash.Hashing; public class KeyBufferTest { @@ -152,20 +152,27 @@ public void testSameKey() { } private void compareKey(byte[] randomKey) { + ByteArrayEntrySerializer serializer = ByteArrayEntrySerializer.ofSize(13); + ByteArrayEntry randomEntry = serializer.randomEntry(randomKey.length); + long entryOffset = NonMemoryPoolHashEntries.ENTRY_OFF_DATA; + long locationOffset = entryOffset + serializer.sizesSize(); + long keyOffset = entryOffset + serializer.entrySize(); - long adr = Uns.allocate(NonMemoryPoolHashEntries.ENTRY_OFF_DATA + randomKey.length, true); + long adr = Uns.allocate(keyOffset + randomKey.length, true); try { KeyBuffer key = new KeyBuffer(randomKey); key.finish(com.oath.halodb.Hasher.create(HashAlgorithm.MURMUR3)); - NonMemoryPoolHashEntries.init(randomKey.length, adr); - Uns.setMemory(adr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA, randomKey.length, (byte) 0); + NonMemoryPoolHashEntries.init(adr); + randomEntry.serializeSizes(adr + entryOffset); + randomEntry.serializeLocation(adr + locationOffset); + Uns.setMemory(adr, keyOffset, randomKey.length, (byte) 0); - assertFalse(key.sameKey(adr)); + assertFalse(SegmentNonMemoryPool.sameKey(randomKey, adr, serializer)); - Uns.copyMemory(randomKey, 0, adr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA, randomKey.length); - NonMemoryPoolHashEntries.init(randomKey.length, adr); - assertTrue(key.sameKey(adr)); + Uns.copyMemory(randomKey, 0, adr, keyOffset, randomKey.length); + NonMemoryPoolHashEntries.init(adr); + assertTrue(SegmentNonMemoryPool.sameKey(randomKey, adr , serializer)); } finally { Uns.free(adr); } diff --git a/src/test/java/com/oath/halodb/LinkedImplTest.java b/src/test/java/com/oath/halodb/LinkedImplTest.java index fdbc735..46b344e 100644 --- a/src/test/java/com/oath/halodb/LinkedImplTest.java +++ b/src/test/java/com/oath/halodb/LinkedImplTest.java @@ -7,38 +7,38 @@ package com.oath.halodb; +import java.io.IOException; + import org.testng.annotations.AfterMethod; import org.testng.annotations.Test; -import java.io.IOException; +public class LinkedImplTest { + static ByteArrayEntrySerializer serializer = ByteArrayEntrySerializer.ofSize(33); -public class LinkedImplTest -{ @AfterMethod(alwaysRun = true) public void deinit() { Uns.clearUnsDebugForTest(); } - static OffHeapHashTable cache() + static OffHeapHashTable cache() { return cache(256); } - static OffHeapHashTable cache(long capacity) + static OffHeapHashTable cache(long capacity) { return cache(capacity, -1); } - static OffHeapHashTable cache(long capacity, int hashTableSize) + static OffHeapHashTable cache(long capacity, int hashTableSize) { return cache(capacity, hashTableSize, -1, -1); } - static OffHeapHashTable cache(long capacity, int hashTableSize, int segments, long maxEntrySize) + static OffHeapHashTable cache(long capacity, int hashTableSize, int segments, long maxEntrySize) { - OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder() - .valueSerializer(HashTableTestUtils.stringSerializer); + OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(serializer); if (hashTableSize > 0) builder.hashTableSize(hashTableSize); if (segments > 0) @@ -53,8 +53,7 @@ static OffHeapHashTable cache(long capacity, int hashTableSize, int segm @Test(expectedExceptions = IllegalArgumentException.class) public void testExtremeHashTableSize() throws IOException { - OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder() - .hashTableSize(1 << 30); + OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(serializer).hashTableSize(1 << 30); builder.build().close(); } diff --git a/src/test/java/com/oath/halodb/MemoryPoolChunkTest.java b/src/test/java/com/oath/halodb/MemoryPoolChunkTest.java index eac7cab..000d824 100644 --- a/src/test/java/com/oath/halodb/MemoryPoolChunkTest.java +++ b/src/test/java/com/oath/halodb/MemoryPoolChunkTest.java @@ -5,17 +5,44 @@ package com.oath.halodb; -import com.google.common.primitives.Longs; +import java.util.Random; import org.testng.Assert; import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.util.Random; +import com.google.common.primitives.Longs; public class MemoryPoolChunkTest { + private final Random r = new Random(); + + private MemoryPoolChunk chunk = null; + + private int chunkSize; + private int fixedKeyLength; + private int fixedEntryLength; + private ByteArrayEntrySerializer serializer; + + private void createChunk() { + serializer = ByteArrayEntrySerializer.ofSize(Math.max(fixedEntryLength - 5, 0)); // uses 2 bytes for key size + chunk = MemoryPoolChunk.create(1, chunkSize, fixedKeyLength, serializer); + } - private MemoryPoolChunk chunk = null; + private byte[] randomKey() { + return HashTableTestUtils.randomBytes(fixedKeyLength); + } + + private ByteArrayEntry randomEntry(int keySize) { + return serializer.randomEntry(keySize); + } + + @BeforeMethod(alwaysRun = true) + private void initParams() { + chunkSize = 1024; + fixedKeyLength = 12; + fixedEntryLength = 20; + } @AfterMethod(alwaysRun = true) private void destroyChunk() { @@ -26,152 +53,207 @@ private void destroyChunk() { @Test public void testSetAndGetMethods() { - int chunkSize = 16 * 1024; - int fixedKeyLength = 12, fixedValueLength = 20; - int slotSize = MemoryPoolHashEntries.HEADER_SIZE + fixedKeyLength + fixedValueLength; + chunkSize = 16 * 1024; - chunk = MemoryPoolChunk.create(chunkSize, fixedKeyLength, fixedValueLength); - int offset = chunk.getWriteOffset(); + createChunk(); + int slotSize = MemoryPoolChunk.slotSize(fixedKeyLength, serializer); + int slots = chunkSize / slotSize; + final int firstOffset = chunk.getWriteOffset(); - Assert.assertEquals(chunk.remaining(), chunkSize); + Assert.assertEquals(chunk.remainingSlots(), chunkSize/slotSize); Assert.assertEquals(chunk.getWriteOffset(), 0); // write to an empty slot. byte[] key = Longs.toByteArray(101); - byte[] value = HashTableTestUtils.randomBytes(fixedValueLength); - MemoryPoolAddress nextAddress = new MemoryPoolAddress((byte) 10, 34343); - chunk.fillNextSlot(key, value, nextAddress); + ByteArrayEntry entry = randomEntry(key.length); + int nextAddress = MemoryPoolAddress.encode(10, 34343); + MemoryPoolChunk.Slot slot = chunk.allocateSlot(); + slot.fillSlot(key, entry, nextAddress); - Assert.assertEquals(chunk.getWriteOffset(), offset + slotSize); - Assert.assertEquals(chunk.remaining(), chunkSize-slotSize); - Assert.assertTrue(chunk.compareKey(offset, key)); - Assert.assertTrue(chunk.compareValue(offset, value)); + Assert.assertEquals(chunk.getWriteOffset(), firstOffset + slotSize); + Assert.assertEquals(chunk.remainingSlots(), slots - 1); + Assert.assertEquals(slot.getKeyLength(), key.length); + Assert.assertTrue(slot.compareFixedKey(key, key.length)); + Assert.assertTrue(slot.compareEntry(entry)); + Assert.assertEquals(slot.readEntry(), entry); - MemoryPoolAddress actual = chunk.getNextAddress(offset); - Assert.assertEquals(actual.chunkIndex, nextAddress.chunkIndex); - Assert.assertEquals(actual.chunkOffset, nextAddress.chunkOffset); + int actual = slot.getNextAddress(); + Assert.assertEquals(actual, nextAddress); // write to the next empty slot. - byte[] key2 = HashTableTestUtils.randomBytes(fixedKeyLength); - byte[] value2 = HashTableTestUtils.randomBytes(fixedValueLength); - MemoryPoolAddress nextAddress2 = new MemoryPoolAddress((byte) 0, 4454545); - chunk.fillNextSlot(key2, value2, nextAddress2); - Assert.assertEquals(chunk.getWriteOffset(), offset + 2*slotSize); - Assert.assertEquals(chunk.remaining(), chunkSize-2*slotSize); - - offset += slotSize; - Assert.assertTrue(chunk.compareKey(offset, key2)); - Assert.assertTrue(chunk.compareValue(offset, value2)); - - actual = chunk.getNextAddress(offset); - Assert.assertEquals(actual.chunkIndex, nextAddress2.chunkIndex); - Assert.assertEquals(actual.chunkOffset, nextAddress2.chunkOffset); + byte[] key2 = randomKey(); + ByteArrayEntry entry2 = randomEntry(key2.length); + int nextAddress2 = MemoryPoolAddress.encode(0, 4454545); + MemoryPoolChunk.Slot slot2 = chunk.allocateSlot(); + slot2.fillSlot(key2, entry2, nextAddress2); + Assert.assertEquals(chunk.getWriteOffset(), firstOffset + 2*slotSize); + Assert.assertEquals(chunk.remainingSlots(), slots - 2); + + Assert.assertEquals(slot2.getKeyLength(), key2.length); + Assert.assertTrue(slot2.compareFixedKey(key2, key2.length)); + Assert.assertTrue(slot2.compareEntry(entry2)); + Assert.assertEquals(slot2.readEntry(), entry2); + + actual = slot2.getNextAddress(); + Assert.assertEquals(actual, nextAddress2); // update an existing slot. byte[] key3 = Longs.toByteArray(0x64735981289L); - byte[] value3 = HashTableTestUtils.randomBytes(fixedValueLength); - MemoryPoolAddress nextAddress3 = new MemoryPoolAddress((byte)-1, -1); - chunk.fillSlot(0, key3, value3, nextAddress3); + ByteArrayEntry entry3 = randomEntry(key3.length); + int nextAddress3 = MemoryPoolAddress.empty; + slot.fillSlot(key3, entry3, nextAddress3); - offset = 0; - Assert.assertTrue(chunk.compareKey(offset, key3)); - Assert.assertTrue(chunk.compareValue(offset, value3)); + Assert.assertEquals(slot.getKeyLength(), key3.length); + Assert.assertTrue(slot.compareFixedKey(key3, key3.length)); + Assert.assertTrue(slot.compareEntry(entry3)); + Assert.assertEquals(slot.readEntry(), entry3); // write offset should remain unchanged. - Assert.assertEquals(chunk.getWriteOffset(), offset + 2*slotSize); - Assert.assertEquals(chunk.remaining(), chunkSize-2*slotSize); + Assert.assertEquals(chunk.getWriteOffset(), firstOffset + 2*slotSize); + Assert.assertEquals(chunk.remainingSlots(), slots - 2); + + Hasher hasher = Hasher.create(HashAlgorithm.MURMUR3); + long keyHash = hasher.hash(key3); + long entryKeyHash = slot.computeFixedKeyHash(hasher, key3.length); + Assert.assertEquals(entryKeyHash, keyHash); + + ByteArrayEntry entry4 = randomEntry(key3.length); + slot.setEntry(entry4); + Assert.assertEquals(slot.readEntry(), entry4); + + long address = Uns.allocate(fixedKeyLength); + try { + slot.copyEntireFixedKey(address); + Uns.compare(address, key3, 0, key3.length); + } finally { + Uns.free(address); + } } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Invalid offset.*") - public void testWithInvalidOffset() { - int chunkSize = 256; - int fixedKeyLength = 100, fixedValueLength = 100; - MemoryPoolAddress next = new MemoryPoolAddress((byte)-1, -1); - chunk = MemoryPoolChunk.create(chunkSize, fixedKeyLength, fixedValueLength); - chunk.fillSlot(chunkSize - 5, HashTableTestUtils.randomBytes(fixedKeyLength), HashTableTestUtils.randomBytes(fixedValueLength), next); + @Test + public void testExtendedSlot() { + createChunk(); + MemoryPoolChunk.Slot slot = chunk.allocateSlot(); + byte[] key = HashTableTestUtils.randomBytes(200); + int next = MemoryPoolAddress.encode(33, 5); + + int writeLen = fixedKeyLength + fixedEntryLength - 7; + slot.fillOverflowSlot(key, key.length - writeLen, writeLen, next); + Assert.assertTrue(slot.compareExtendedKey(key, key.length - writeLen, writeLen)); + + long address = Uns.allocate(writeLen); + try { + slot.copyExtendedKey(address, 0, writeLen); + Uns.compare(address, key, key.length - writeLen, writeLen); + } finally { + Uns.free(address); + } } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Invalid request. Key length.*") - public void testWithInvalidKey() { - int chunkSize = 256; - int fixedKeyLength = 32, fixedValueLength = 100; - MemoryPoolAddress next = new MemoryPoolAddress((byte)-1, -1); - chunk = MemoryPoolChunk.create(chunkSize, fixedKeyLength, fixedValueLength); - chunk.fillSlot(chunkSize - 5, HashTableTestUtils.randomBytes(fixedKeyLength + 10), HashTableTestUtils.randomBytes(fixedValueLength), next); + @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Invalid request.*") + public void testWithInvalidSlot() { + chunkSize = 256; + fixedKeyLength = 100; + fixedEntryLength = 100; + createChunk(); + chunk.slotFor(999); } @Test public void testCompare() { - int chunkSize = 1024; - int fixedKeyLength = 9, fixedValueLength = 15; + chunkSize = 1024; + fixedKeyLength = 9; + fixedEntryLength = 15; + createChunk(); - chunk = MemoryPoolChunk.create(chunkSize, fixedKeyLength, fixedValueLength); - byte[] key = HashTableTestUtils.randomBytes(fixedKeyLength); - byte[] value = HashTableTestUtils.randomBytes(fixedValueLength); - int offset = 0; - chunk.fillSlot(offset, key, value, new MemoryPoolAddress((byte)-1, -1)); + byte[] key = randomKey(); + ByteArrayEntry entry = randomEntry(key.length); + MemoryPoolChunk.Slot slot = chunk.allocateSlot(); + slot.fillSlot(key, entry, MemoryPoolAddress.empty); - Assert.assertTrue(chunk.compareKey(offset, key)); - Assert.assertTrue(chunk.compareValue(offset, value)); + Assert.assertEquals(slot.getKeyLength(), key.length); + Assert.assertTrue(slot.compareFixedKey(key, key.length)); + Assert.assertTrue(slot.compareEntry(entry)); byte[] smallKey = new byte[key.length-1]; System.arraycopy(key, 0, smallKey, 0, smallKey.length); - Assert.assertFalse(chunk.compareKey(offset, smallKey)); + Assert.assertNotEquals(slot.getKeyLength(), smallKey.length); key[fixedKeyLength-1] = (byte)~key[fixedKeyLength-1]; - Assert.assertFalse(chunk.compareKey(offset, key)); + Assert.assertEquals(slot.getKeyLength(), key.length); + Assert.assertFalse(slot.compareFixedKey(key, key.length)); - value[0] = (byte)~value[0]; - Assert.assertFalse(chunk.compareValue(offset, value)); + entry.bytes[0] = (byte)~entry.bytes[0]; + Assert.assertFalse(slot.compareEntry(entry)); } @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Invalid request.*") public void testCompareKeyWithException() { - int chunkSize = 1024; - Random r = new Random(); - int fixedKeyLength = 1 + r.nextInt(100), fixedValueLength = 1 + r.nextInt(100); - - chunk = MemoryPoolChunk.create(chunkSize, fixedKeyLength, fixedValueLength); - byte[] key = HashTableTestUtils.randomBytes(fixedKeyLength); - byte[] value = HashTableTestUtils.randomBytes(fixedValueLength); - int offset = 0; - chunk.fillSlot(offset, key, value, new MemoryPoolAddress((byte)-1, -1)); - - byte[] bigKey = HashTableTestUtils.randomBytes(fixedKeyLength + 1); - chunk.compareKey(offset, bigKey); + chunkSize = 1024; + fixedKeyLength = r.nextInt(100); + fixedEntryLength = r.nextInt(100); + createChunk(); + byte[] key = randomKey(); + ByteArrayEntry entry = randomEntry(key.length); + MemoryPoolChunk.Slot slot = chunk.allocateSlot(); + slot.fillSlot(key, entry, MemoryPoolAddress.empty); + byte[] bigKey = HashTableTestUtils.randomBytes(fixedKeyLength + 1); + slot.compareFixedKey(bigKey, bigKey.length); } @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Invalid request.*") - public void testCompareValueWithException() { - int chunkSize = 1024; - Random r = new Random(); - int fixedKeyLength = 1 + r.nextInt(100), fixedValueLength = 1 + r.nextInt(100); - - chunk = MemoryPoolChunk.create(chunkSize, fixedKeyLength, fixedValueLength); - byte[] key = HashTableTestUtils.randomBytes(fixedKeyLength); - byte[] value = HashTableTestUtils.randomBytes(fixedValueLength); - int offset = 0; - chunk.fillSlot(offset, key, value, new MemoryPoolAddress((byte)-1, -1)); - - byte[] bigValue = HashTableTestUtils.randomBytes(fixedValueLength + 1); - chunk.compareValue(offset, bigValue); + public void testCompareExtendedKeyWithException() { + chunkSize = 1024; + fixedKeyLength = r.nextInt(100); + fixedEntryLength = r.nextInt(100); + + createChunk(); + byte[] key = randomKey(); + MemoryPoolChunk.Slot slot = chunk.allocateSlot(); + slot.compareExtendedKey(key, 0, 300); } @Test public void setAndGetNextAddress() { - int chunkSize = 1024; - Random r = new Random(); - int fixedKeyLength = 1 + r.nextInt(100), fixedValueLength = 1 + r.nextInt(100); + chunkSize = 1024; + fixedKeyLength = r.nextInt(100); + fixedEntryLength = r.nextInt(100); + + createChunk(); + + int slotSize = MemoryPoolChunk.slotSize(fixedKeyLength, serializer); + int numSlots = chunkSize / slotSize; - chunk = MemoryPoolChunk.create(chunkSize, fixedKeyLength, fixedValueLength); + int nextAddress = MemoryPoolAddress.encode((r.nextInt(255) + 1), r.nextInt(numSlots)); + MemoryPoolChunk.Slot slot = chunk.slotFor(r.nextInt(numSlots)); - MemoryPoolAddress nextAddress = new MemoryPoolAddress((byte)r.nextInt(Byte.MAX_VALUE), r.nextInt()); - int offset = r.nextInt(chunkSize - fixedKeyLength - fixedValueLength - MemoryPoolHashEntries.HEADER_SIZE); - chunk.setNextAddress(offset, nextAddress); + slot.setNextAddress(nextAddress); - Assert.assertEquals(chunk.getNextAddress(offset), nextAddress); + Assert.assertEquals(slot.getNextAddress(), nextAddress); + } + + @Test + public void testInvalidSlotSize() { + MemoryPoolAddress.encode(0, MemoryPoolAddress.MAX_NUMBER_OF_SLOTS); + try { + MemoryPoolAddress.encode(0, MemoryPoolAddress.MAX_NUMBER_OF_SLOTS + 1); + Assert.fail("MemoryPoolAddress should throw when attempting to encode an invalid slot size"); + } catch (IllegalArgumentException expected){ + // nothing + } + } + @Test + public void testInvalidChunkEncode() { + MemoryPoolAddress.encode(255, 0); + try { + MemoryPoolAddress.encode(256, 0); + Assert.fail("MemoryPoolAddress should throw when attempting to encode an invalid chunk id"); + } catch (IllegalArgumentException expected){ + // nothing + } } + } diff --git a/src/test/java/com/oath/halodb/NonMemoryPoolHashEntriesTest.java b/src/test/java/com/oath/halodb/NonMemoryPoolHashEntriesTest.java index 1292c31..1dc28d2 100644 --- a/src/test/java/com/oath/halodb/NonMemoryPoolHashEntriesTest.java +++ b/src/test/java/com/oath/halodb/NonMemoryPoolHashEntriesTest.java @@ -7,13 +7,11 @@ package com.oath.halodb; +import static org.testng.Assert.assertEquals; + import org.testng.annotations.AfterMethod; import org.testng.annotations.Test; -import java.nio.ByteBuffer; - -import static org.testng.Assert.*; - public class NonMemoryPoolHashEntriesTest { @AfterMethod(alwaysRun = true) @@ -30,46 +28,11 @@ public void testInit() throws Exception long adr = Uns.allocate(MIN_ALLOC_LEN); try { - NonMemoryPoolHashEntries.init(5, adr); + NonMemoryPoolHashEntries.init(adr); assertEquals(Uns.getLong(adr, NonMemoryPoolHashEntries.ENTRY_OFF_NEXT), 0L); - assertEquals(Uns.getByte(adr, NonMemoryPoolHashEntries.ENTRY_OFF_KEY_LENGTH), 5); assertEquals(NonMemoryPoolHashEntries.getNext(adr), 0L); - assertEquals(NonMemoryPoolHashEntries.getKeyLen(adr), 5L); - } - finally - { - Uns.free(adr); - } - } - - @Test - public void testCompareKey() throws Exception - { - long adr = Uns.allocate(MIN_ALLOC_LEN); - try - { - NonMemoryPoolHashEntries.init(11, adr); - - ByteBuffer buffer = ByteBuffer.allocate(11); - buffer.putInt(0x98765432); - buffer.putInt(0xabcdabba); - buffer.put((byte)(0x44 & 0xff)); - buffer.put((byte)(0x55 & 0xff)); - buffer.put((byte)(0x88 & 0xff)); - - KeyBuffer key = new KeyBuffer(buffer.array()); - key.finish(Hasher.create(HashAlgorithm.MURMUR3)); - - Uns.setMemory(adr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA, 11, (byte) 0); - - assertFalse(key.sameKey(adr)); - - Uns.copyMemory(key.buffer, 0, adr, NonMemoryPoolHashEntries.ENTRY_OFF_DATA, 11); - NonMemoryPoolHashEntries.init(11, adr); - - assertTrue(key.sameKey(adr)); } finally { @@ -84,13 +47,13 @@ public void testGetSetNext() throws Exception try { Uns.setMemory(adr, 0, MIN_ALLOC_LEN, (byte) 0); - NonMemoryPoolHashEntries.init(5, adr); Uns.putLong(adr, NonMemoryPoolHashEntries.ENTRY_OFF_NEXT, 0x98765432abdffeedL); assertEquals(NonMemoryPoolHashEntries.getNext(adr), 0x98765432abdffeedL); NonMemoryPoolHashEntries.setNext(adr, 0xfafefcfb23242526L); assertEquals(Uns.getLong(adr, NonMemoryPoolHashEntries.ENTRY_OFF_NEXT), 0xfafefcfb23242526L); + assertEquals(NonMemoryPoolHashEntries.getNext(adr), 0xfafefcfb23242526L); } finally { diff --git a/src/test/java/com/oath/halodb/OffHeapHashTableBuilderTest.java b/src/test/java/com/oath/halodb/OffHeapHashTableBuilderTest.java index c93371f..58d8df3 100644 --- a/src/test/java/com/oath/halodb/OffHeapHashTableBuilderTest.java +++ b/src/test/java/com/oath/halodb/OffHeapHashTableBuilderTest.java @@ -8,19 +8,16 @@ package com.oath.halodb; import org.testng.Assert; -import org.testng.annotations.AfterMethod; import org.testng.annotations.Test; -import java.nio.ByteBuffer; -import java.util.HashSet; - public class OffHeapHashTableBuilderTest { - + ByteArrayEntrySerializer serializer = ByteArrayEntrySerializer.ofSize(3); + @Test public void testHashTableSize() throws Exception { - OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(); + OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(serializer); Assert.assertEquals(builder.getHashTableSize(), 8192); builder.hashTableSize(12345); Assert.assertEquals(builder.getHashTableSize(), 12345); @@ -29,7 +26,7 @@ public void testHashTableSize() throws Exception @Test public void testChunkSize() throws Exception { - OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(); + OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(serializer); builder.memoryPoolChunkSize(12345); Assert.assertEquals(builder.getMemoryPoolChunkSize(), 12345); } @@ -42,7 +39,7 @@ public void testSegmentCount() throws Exception while (Integer.bitCount(segments) != 1) segments++; - OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(); + OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(serializer); Assert.assertEquals(builder.getSegmentCount(), segments); builder.segmentCount(12345); Assert.assertEquals(builder.getSegmentCount(), 12345); @@ -51,7 +48,7 @@ public void testSegmentCount() throws Exception @Test public void testLoadFactor() throws Exception { - OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(); + OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(serializer); Assert.assertEquals(builder.getLoadFactor(), .75f); builder.loadFactor(12345); Assert.assertEquals(builder.getLoadFactor(), 12345.0f); @@ -60,33 +57,13 @@ public void testLoadFactor() throws Exception @Test public void testValueSerializer() throws Exception { - OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(); - Assert.assertNull(builder.getValueSerializer()); - - HashTableValueSerializer inst = new HashTableValueSerializer() - { - public void serialize(String s, ByteBuffer out) - { - - } - - public String deserialize(ByteBuffer in) - { - return null; - } - - public int serializedSize(String s) - { - return 0; - } - }; - builder.valueSerializer(inst); - Assert.assertSame(builder.getValueSerializer(), inst); + OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(serializer); + Assert.assertSame(builder.getEntrySerializer(), serializer); } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = ".*Need to set fixedValueSize*") + @Test public void testFixedValueSize() throws Exception { - OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(); + OffHeapHashTableBuilder builder = OffHeapHashTableBuilder.newBuilder(serializer); builder.build(); } } diff --git a/src/test/java/com/oath/halodb/RecordTest.java b/src/test/java/com/oath/halodb/RecordTest.java index da2b829..36f2766 100644 --- a/src/test/java/com/oath/halodb/RecordTest.java +++ b/src/test/java/com/oath/halodb/RecordTest.java @@ -5,56 +5,56 @@ package com.oath.halodb; -import org.testng.Assert; -import org.testng.annotations.Test; - import java.nio.ByteBuffer; import java.util.zip.CRC32; +import org.testng.Assert; +import org.testng.annotations.Test; + public class RecordTest { @Test public void testSerializeHeader() { - byte keySize = 8; + int keySize = 300; // 256 plus 44 int valueSize = 100; long sequenceNumber = 34543434343L; - int version = 128; + byte version = 29; - Record.Header header = new Record.Header(0, version, keySize, valueSize, sequenceNumber); + RecordEntry.Header header = new RecordEntry.Header(0, version, keySize, valueSize, sequenceNumber); ByteBuffer serialized = header.serialize(); - Assert.assertEquals(keySize, serialized.get(Record.Header.KEY_SIZE_OFFSET)); - Assert.assertEquals(valueSize, serialized.getInt(Record.Header.VALUE_SIZE_OFFSET)); - Assert.assertEquals(sequenceNumber, serialized.getLong(Record.Header.SEQUENCE_NUMBER_OFFSET)); - Assert.assertEquals(Utils.toUnsignedByte(serialized.get(Record.Header.VERSION_OFFSET)), version); + Assert.assertEquals(serialized.get(RecordEntry.Header.KEY_SIZE_OFFSET) & 0xFF, keySize & 0xFF); + Assert.assertEquals(serialized.getInt(RecordEntry.Header.VALUE_SIZE_OFFSET), valueSize); + Assert.assertEquals(serialized.getLong(RecordEntry.Header.SEQUENCE_NUMBER_OFFSET), sequenceNumber); + Assert.assertEquals(serialized.get(RecordEntry.Header.VERSION_OFFSET) & 0xFF, (version << 3) | (keySize >>> 8)); } @Test public void testDeserializeHeader() { long checkSum = 23434; - byte keySize = 8; + int keySize = 200; int valueSize = 100; long sequenceNumber = 34543434343L; int version = 2; - ByteBuffer buffer = ByteBuffer.allocate(Record.Header.HEADER_SIZE); + ByteBuffer buffer = ByteBuffer.allocate(RecordEntry.Header.HEADER_SIZE); buffer.putInt(Utils.toSignedIntFromLong(checkSum)); - buffer.put((byte)version); - buffer.put(keySize); + buffer.put((byte)(version << 3)); + buffer.put((byte)keySize); buffer.putInt(valueSize); buffer.putLong(sequenceNumber); buffer.flip(); - Record.Header header = Record.Header.deserialize(buffer); + RecordEntry.Header header = RecordEntry.Header.deserialize(buffer); Assert.assertEquals(checkSum, header.getCheckSum()); Assert.assertEquals(version, header.getVersion()); Assert.assertEquals(keySize, header.getKeySize()); Assert.assertEquals(valueSize, header.getValueSize()); Assert.assertEquals(sequenceNumber, header.getSequenceNumber()); - Assert.assertEquals(keySize + valueSize + Record.Header.HEADER_SIZE, header.getRecordSize()); + Assert.assertEquals(keySize + valueSize + RecordEntry.Header.HEADER_SIZE, header.getRecordSize()); } @Test @@ -62,21 +62,19 @@ public void testSerializeRecord() { byte[] key = TestUtils.generateRandomByteArray(); byte[] value = TestUtils.generateRandomByteArray(); long sequenceNumber = 192; - int version = 13; + byte version = 13; - Record record = new Record(key, value); - record.setSequenceNumber(sequenceNumber); - record.setVersion(version); + RecordEntry.Header header = new RecordEntry.Header(0, version, key.length, value.length, sequenceNumber); + RecordEntry record = new RecordEntry(header, key, value); ByteBuffer[] buffers = record.serialize(); CRC32 crc32 = new CRC32(); - crc32.update(buffers[0].array(), Record.Header.VERSION_OFFSET, buffers[0].array().length - Record.Header.CHECKSUM_SIZE); + crc32.update(buffers[0].array(), RecordEntry.Header.VERSION_OFFSET, buffers[0].array().length - RecordEntry.Header.CHECKSUM_SIZE); crc32.update(key); crc32.update(value); - Record.Header header = new Record.Header(0, version, (byte)key.length, value.length, sequenceNumber); ByteBuffer headerBuf = header.serialize(); - headerBuf.putInt(Record.Header.CHECKSUM_OFFSET, Utils.toSignedIntFromLong(crc32.getValue())); + headerBuf.putInt(RecordEntry.Header.CHECKSUM_OFFSET, Utils.toSignedIntFromLong(crc32.getValue())); Assert.assertEquals(headerBuf, buffers[0]); Assert.assertEquals(ByteBuffer.wrap(key), buffers[1]); diff --git a/src/test/java/com/oath/halodb/RehashTest.java b/src/test/java/com/oath/halodb/RehashTest.java index 91eae6e..ce8ab9e 100644 --- a/src/test/java/com/oath/halodb/RehashTest.java +++ b/src/test/java/com/oath/halodb/RehashTest.java @@ -7,15 +7,15 @@ package com.oath.halodb; -import com.google.common.primitives.Longs; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +import java.io.IOException; import org.testng.annotations.AfterMethod; import org.testng.annotations.Test; -import java.io.IOException; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertTrue; +import com.google.common.primitives.Longs; public class RehashTest { @@ -28,22 +28,21 @@ public void deinit() @Test public void testRehash() throws IOException { - try (OffHeapHashTable cache = OffHeapHashTableBuilder.newBuilder() - .valueSerializer(HashTableTestUtils.byteArraySerializer) + ByteArrayEntrySerializer serializer = ByteArrayEntrySerializer.ofSize(8); + try (OffHeapHashTable cache = OffHeapHashTableBuilder.newBuilder(serializer) .hashTableSize(64) .segmentCount(4) - .fixedValueSize(8) .build()) { for (int i = 0; i < 100000; i++) - cache.put(Longs.toByteArray(i), Longs.toByteArray(i)); + cache.put(Longs.toByteArray(i), serializer.createEntry(8, Longs.toByteArray(i))); assertTrue(cache.stats().getRehashCount() > 0); for (int i = 0; i < 100000; i++) { - byte[] v = cache.get(Longs.toByteArray(i)); - assertEquals(Longs.fromByteArray(v), i); + ByteArrayEntry v = cache.get(Longs.toByteArray(i)); + assertEquals(Longs.fromByteArray(v.bytes), i); } } } diff --git a/src/test/java/com/oath/halodb/SegmentWithMemoryPoolTest.java b/src/test/java/com/oath/halodb/SegmentWithMemoryPoolTest.java index b6f570f..71beee1 100644 --- a/src/test/java/com/oath/halodb/SegmentWithMemoryPoolTest.java +++ b/src/test/java/com/oath/halodb/SegmentWithMemoryPoolTest.java @@ -5,48 +5,208 @@ package com.oath.halodb; -import com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.stream.Stream; import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; +import com.google.common.collect.Lists; public class SegmentWithMemoryPoolTest { - @Test - public void testChunkAllocations() { + Hasher hasher = Hasher.create(HashAlgorithm.MURMUR3); + ByteArrayEntrySerializer serializer = ByteArrayEntrySerializer.ofSize(16); - int fixedKeySize = 8; - int fixedValueSize = 18; - int noOfEntries = 100; - int noOfChunks = 2; - int fixedSlotSize = MemoryPoolHashEntries.HEADER_SIZE + fixedKeySize + fixedValueSize; + int fixedKeySize; + int noOfEntries; + int noOfChunks; + int fixedSlotSize; - OffHeapHashTableBuilder builder = OffHeapHashTableBuilder - .newBuilder() + SegmentWithMemoryPool segment = null; + + private OffHeapHashTableBuilder builder() { + return OffHeapHashTableBuilder + .newBuilder(serializer) .fixedKeySize(fixedKeySize) - .fixedValueSize(fixedValueSize) - // chunkSize set such that noOfEntries/2 can set filled in one. - .memoryPoolChunkSize(noOfEntries/noOfChunks * fixedSlotSize) - .valueSerializer(HashTableTestUtils.byteArraySerializer); + // chunkSize set such that noOfEntries/2 can set filled in one. + .memoryPoolChunkSize(noOfEntries/noOfChunks * fixedSlotSize); + } - SegmentWithMemoryPool segment = new SegmentWithMemoryPool<>(builder); + @BeforeMethod(alwaysRun = true) + public void initialize() { + fixedKeySize = 8; + noOfEntries = 100; + noOfChunks = 2; + fixedSlotSize = MemoryPoolChunk.slotSize(fixedKeySize, serializer); + } - addEntriesToSegment(segment, Hasher.create(HashAlgorithm.MURMUR3), noOfEntries, fixedKeySize, fixedValueSize); + @AfterMethod(alwaysRun = true) + public void releaseSegment() { + if (segment != null) { + segment.release(); + } + } - // each chunk can hold noOfEntries/2 hence two chunks must be allocated. - Assert.assertEquals(segment.numberOfChunks(), 2); - Assert.assertEquals(segment.numberOfSlots(), noOfEntries); - Assert.assertEquals(segment.size(), noOfEntries); - Assert.assertEquals(segment.putAddCount(), noOfEntries); + @Test + public void testTinyKeys() { + segment = new SegmentWithMemoryPool<>(builder()); + Record rec = createRecord(0); + validateBasicPutGet(rec, 1); + Record rec2 = createRecord(1); + validateBasicPutGet(rec2, 1); + } + + @Test + public void testSmallKeys() { + segment = new SegmentWithMemoryPool<>(builder()); + Record rec = createRecord(fixedKeySize); + validateBasicPutGet(rec, 1); + Record rec2 = createRecord(fixedKeySize -1); + validateBasicPutGet(rec2, 1); + } + + @Test + public void testLargeKeys() { + segment = new SegmentWithMemoryPool<>(builder()); + Record rec = createRecord(fixedKeySize + 1); + validateBasicPutGet(rec, 2); + Record rec2 = createRecord(fixedKeySize + 15); + validateBasicPutGet(rec2, 2); + Record rec3 = createRecord(fixedKeySize + fixedKeySize + serializer.entrySize() - 1); + validateBasicPutGet(rec3, 2); + Record rec4 = createRecord(fixedKeySize + fixedKeySize + serializer.entrySize()); + validateBasicPutGet(rec4, 2); + } + + @Test + public void testHugeKeys() { + segment = new SegmentWithMemoryPool<>(builder()); + int fillsFourSlots = (fixedKeySize * 4) + (3 * serializer.entrySize()); + int barelyUsesFiveSlots = fillsFourSlots + 1; + int nearlyUsesFourSlots = fillsFourSlots - 1; + Record rec = createRecord(fillsFourSlots); + validateBasicPutGet(rec, 4); + rec = createRecord(barelyUsesFiveSlots); + validateBasicPutGet(rec, 5); + rec = createRecord(nearlyUsesFourSlots); + validateBasicPutGet(rec, 4); + + Record max = createRecord(2047); + int expectedSlotsUsed = 2 + ((2047 - fixedKeySize - 1) / (fixedKeySize + serializer.entrySize())); + validateBasicPutGet(max, expectedSlotsUsed); + } + + private void validateBasicPutGet(Record rec, int expectedSlotsPerEntry) { + KeyBuffer key = rec.keyBuffer; + ByteArrayEntry entry = rec.entry; + + long initialSize = segment.size(); + int initialFreeListHead = segment.getFreeListHead(); + long initialFreeListSize = segment.freeListSize(); + long initialPutAddCount = segment.putAddCount(); + long initialPutReplaceCount = segment.putReplaceCount(); + long initialRemoveCount = segment.removeCount(); + long initialHitCount = segment.hitCount(); + long initialMissCount = segment.missCount(); + + // put when not present, but old value exists and doesn't match, so this should not add + Assert.assertFalse(segment.putEntry(key, entry, false, entry)); + Assert.assertFalse(segment.containsEntry(key)); + Assert.assertNull(segment.getEntry(key)); + Assert.assertEquals(segment.size(), initialSize); + Assert.assertEquals(segment.freeListSize(), initialFreeListSize); + Assert.assertEquals(segment.putAddCount(), initialPutAddCount); + Assert.assertEquals(segment.putReplaceCount(), initialPutReplaceCount); + Assert.assertEquals(segment.getFreeListHead(), initialFreeListHead); + Assert.assertEquals(segment.removeCount(), initialRemoveCount); + Assert.assertEquals(segment.hitCount(), initialHitCount); + Assert.assertEquals(segment.missCount(), initialMissCount + 2); + + // put with putIfAbsent = false should add -- it only disables overwriting existing + Assert.assertTrue(segment.putEntry(key, entry, false, null)); + Assert.assertTrue(segment.containsEntry(key)); + Assert.assertEquals(segment.getEntry(key), entry); + Assert.assertEquals(segment.size(), initialSize + 1); + Assert.assertEquals(segment.putAddCount(), initialPutAddCount + 1); + Assert.assertEquals(segment.putReplaceCount(), initialPutReplaceCount); + Assert.assertEquals(segment.freeListSize(), Math.max(initialFreeListSize - expectedSlotsPerEntry, 0)); + if(MemoryPoolAddress.isEmpty(initialFreeListHead)) { + Assert.assertEquals(segment.freeListSize(), initialFreeListSize); + } else { + Assert.assertNotEquals(segment.getFreeListHead(), initialFreeListHead); + } + Assert.assertEquals(segment.removeCount(), initialRemoveCount); + Assert.assertEquals(segment.hitCount(), initialHitCount + 2); + Assert.assertEquals(segment.missCount(), initialMissCount + 2); + + // put again over existing value while putIfAbsent is true should not overwrite + Assert.assertFalse(segment.putEntry(key, entry, true, null)); + Assert.assertEquals(segment.putReplaceCount(), initialPutReplaceCount); + Assert.assertEquals(segment.getEntry(key), entry); + Assert.assertEquals(segment.size(), initialSize + 1); + + // put again over existing value with a non-matching oldValue will not overwrite + ByteArrayEntry entry2 = serializer.randomEntry(key.size()); + Assert.assertFalse(segment.putEntry(key, entry2, false, entry2)); + Assert.assertEquals(segment.putReplaceCount(), initialPutReplaceCount); + Assert.assertEquals(segment.getEntry(key), entry); + Assert.assertEquals(segment.size(), initialSize + 1); + + // put again with different value and matching oldValue should overwrite + Assert.assertTrue(segment.putEntry(key, entry2, false, entry)); + Assert.assertEquals(segment.putReplaceCount(), initialPutReplaceCount + 1); + Assert.assertEquals(segment.getEntry(key), entry2); + Assert.assertEquals(segment.size(), initialSize + 1); + Assert.assertEquals(segment.freeListSize(), Math.max(initialFreeListSize - expectedSlotsPerEntry, 0)); + if(MemoryPoolAddress.isEmpty(initialFreeListHead)) { + Assert.assertEquals(segment.getFreeListHead(), initialFreeListHead); + } else { + Assert.assertNotEquals(segment.getFreeListHead(), initialFreeListHead); + } + Assert.assertEquals(segment.hitCount(), initialHitCount + 5); + Assert.assertEquals(segment.missCount(), initialMissCount + 2); + + // remove + Assert.assertTrue(segment.removeEntry(key)); + Assert.assertFalse(segment.containsEntry(key)); + Assert.assertNull(segment.getEntry(key)); + Assert.assertFalse(segment.removeEntry(key)); + Assert.assertEquals(segment.size(), initialSize); + Assert.assertEquals(segment.putAddCount(), initialPutAddCount + 1); + Assert.assertEquals(segment.putReplaceCount(), initialPutReplaceCount + 1); + Assert.assertEquals(segment.freeListSize(), Math.max(initialFreeListSize, expectedSlotsPerEntry)); + if(MemoryPoolAddress.isEmpty(initialFreeListHead)) { + Assert.assertNotEquals(segment.getFreeListHead(), initialFreeListHead); + } else { + Assert.assertEquals(segment.getFreeListHead(), initialFreeListHead); + } + Assert.assertEquals(segment.removeCount(), initialRemoveCount + 1); + Assert.assertEquals(segment.hitCount(), initialHitCount + 5); + Assert.assertEquals(segment.missCount(), initialMissCount + 4); + + } + + @Test + public void testChunkAllocations() { + segment = new SegmentWithMemoryPool<>(builder()); + + addEntriesToSegment(noOfEntries); + int noLargeEntries = noOfEntries / 4; + addLargeEntriesToSegment(noLargeEntries); // will add an equal number of slots as above, but 1/4 the entries + + // each chunk can hold noOfEntries/2 slots hence four chunks must be allocated. + Assert.assertEquals(segment.numberOfChunks(), 4); + Assert.assertEquals(segment.numberOfSlots(), noOfEntries * 2); + Assert.assertEquals(segment.size(), noOfEntries + noLargeEntries); + Assert.assertEquals(segment.putAddCount(), noOfEntries + noLargeEntries); Assert.assertEquals(segment.freeListSize(), 0); - // All slots in chunk should be written to. + // All slots in chunk should be written to and full. for (int i = 0; i < segment.numberOfChunks(); i++) { Assert.assertEquals(segment.getChunkWriteOffset(i), noOfEntries/noOfChunks * fixedSlotSize); } @@ -54,144 +214,145 @@ public void testChunkAllocations() { @Test public void testFreeList() { - int fixedKeySize = 8; - int fixedValueSize = 18; - int noOfEntries = 100; - int chunkCount = 2; - int fixedSlotSize = MemoryPoolHashEntries.HEADER_SIZE + fixedKeySize + fixedValueSize; - MemoryPoolAddress emptyList = new MemoryPoolAddress((byte) -1, -1); - - OffHeapHashTableBuilder builder = OffHeapHashTableBuilder - .newBuilder() - .fixedKeySize(fixedKeySize) - .fixedValueSize(fixedValueSize) - // chunkSize set such that noOfEntries/2 can set filled in one. - .memoryPoolChunkSize(noOfEntries / chunkCount * fixedSlotSize) - .valueSerializer(HashTableTestUtils.byteArraySerializer); - - SegmentWithMemoryPool segment = new SegmentWithMemoryPool<>(builder); + segment = new SegmentWithMemoryPool<>(builder()); //Add noOfEntries to the segment. This should require chunks. - Hasher hasher = Hasher.create(HashAlgorithm.MURMUR3); - List records = addEntriesToSegment(segment, hasher, noOfEntries, fixedKeySize, fixedValueSize); - - // each chunk can hold noOfEntries/2 hence two chunks must be allocated. - Assert.assertEquals(segment.numberOfChunks(), chunkCount); - Assert.assertEquals(segment.size(), noOfEntries); - Assert.assertEquals(segment.putAddCount(), noOfEntries); + List records = addEntriesToSegment(noOfEntries); + // 1/4 the entries, but the same number of slots used + int noLargeEntries = noOfEntries / 4; + int totalEntries = noOfEntries + noLargeEntries; + int totalSlots = noOfEntries + (4 * noLargeEntries); + List bigRecords = addLargeEntriesToSegment(noLargeEntries); + + // each chunk can hold noOfEntries/2 hence four chunks must be allocated. + Assert.assertEquals(segment.numberOfChunks(), noOfChunks * 2); + Assert.assertEquals(segment.size(), totalEntries); + Assert.assertEquals(segment.putAddCount(), totalEntries); Assert.assertEquals(segment.freeListSize(), 0); - Assert.assertEquals(segment.getFreeListHead(), emptyList); + Assert.assertTrue(MemoryPoolAddress.isEmpty(segment.getFreeListHead())); // remove all entries from the segment - // all slots should now be part of the free list. + // all slots should now be part of the free list. Lists.reverse(records).forEach(k -> segment.removeEntry(k.keyBuffer)); + Lists.reverse(bigRecords).forEach(k -> segment.removeEntry(k.keyBuffer)); - Assert.assertEquals(segment.freeListSize(), noOfEntries); - Assert.assertNotEquals(segment.getFreeListHead(), emptyList); - Assert.assertEquals(segment.removeCount(), noOfEntries); + Assert.assertEquals(segment.freeListSize(), totalSlots); + Assert.assertFalse(MemoryPoolAddress.isEmpty(segment.getFreeListHead())); + Assert.assertEquals(segment.removeCount(), totalEntries); Assert.assertEquals(segment.size(), 0); // Add noOfEntries to the segment. - // All entries must be added to slots part of the freelist. - records = addEntriesToSegment(segment, hasher, noOfEntries, fixedKeySize, fixedValueSize); + // All entries must be added to slots part of the freelist. + records = addEntriesToSegment(noOfEntries); + bigRecords = addLargeEntriesToSegment(noLargeEntries); - Assert.assertEquals(segment.numberOfChunks(), chunkCount); - Assert.assertEquals(segment.size(), noOfEntries); + Assert.assertEquals(segment.numberOfChunks(), noOfChunks * 2); + Assert.assertEquals(segment.size(), totalEntries); Assert.assertEquals(segment.freeListSize(), 0); // after all slots in the free list are used head should point to - // an empty list. - Assert.assertEquals(segment.getFreeListHead(), emptyList); + // an empty list. + Assert.assertTrue(MemoryPoolAddress.isEmpty(segment.getFreeListHead())); // remove only some of the elements. Random r = new Random(); int elementsRemoved = 0; + int bigElementsRemoved = 0; for (int i = 0; i < noOfEntries/3; i++) { if(segment.removeEntry(records.get(r.nextInt(records.size())).keyBuffer)) elementsRemoved++; } + for (int i = 0; i < noLargeEntries/3; i++) { + if(segment.removeEntry(bigRecords.get(r.nextInt(bigRecords.size())).keyBuffer)) + bigElementsRemoved++; + } - Assert.assertEquals(segment.freeListSize(), elementsRemoved); - Assert.assertNotEquals(segment.getFreeListHead(), emptyList); - Assert.assertEquals(segment.size(), noOfEntries-elementsRemoved); + Assert.assertEquals(segment.freeListSize(), elementsRemoved + (bigElementsRemoved * 4)); + Assert.assertFalse(MemoryPoolAddress.isEmpty(segment.getFreeListHead())); + Assert.assertEquals(segment.size(), totalEntries - (elementsRemoved + bigElementsRemoved)); // add removed elements back. - addEntriesToSegment(segment, hasher, elementsRemoved, fixedKeySize, fixedValueSize); + addEntriesToSegment(elementsRemoved); + addLargeEntriesToSegment(bigElementsRemoved); - Assert.assertEquals(segment.numberOfChunks(), chunkCount); - Assert.assertEquals(segment.size(), noOfEntries); + Assert.assertEquals(segment.numberOfChunks(), noOfChunks * 2); + Assert.assertEquals(segment.size(), totalEntries); Assert.assertEquals(segment.freeListSize(), 0); - Assert.assertEquals(segment.getFreeListHead(), emptyList); + Assert.assertTrue(MemoryPoolAddress.isEmpty(segment.getFreeListHead())); } - @Test(expectedExceptions = OutOfMemoryError.class, expectedExceptionsMessageRegExp = "Each segment can have at most 128 chunks.") + @Test(expectedExceptions = OutOfMemoryError.class, expectedExceptionsMessageRegExp = "Each segment can have at most 255 chunks.") public void testOutOfMemoryException() { - int fixedKeySize = 8; - int fixedValueSize = 18; - int fixedSlotSize = MemoryPoolHashEntries.HEADER_SIZE + fixedKeySize + fixedValueSize; - - // Each segment can have only Byte.MAX_VALUE chunks. + // Each segment can have only 255 chunks. // we add more that that. - int noOfEntries = Byte.MAX_VALUE * 2; + noOfEntries = 255 + 1; - OffHeapHashTableBuilder builder = OffHeapHashTableBuilder - .newBuilder() - .fixedKeySize(fixedKeySize) - .fixedValueSize(fixedValueSize) - // chunkSize set so that each can contain only one entry. - .memoryPoolChunkSize(fixedSlotSize) - .valueSerializer(HashTableTestUtils.byteArraySerializer); + segment = new SegmentWithMemoryPool<>(builder().memoryPoolChunkSize(fixedSlotSize)); + addEntriesToSegment(noOfEntries); + } - SegmentWithMemoryPool segment = new SegmentWithMemoryPool<>(builder); - addEntriesToSegment(segment, Hasher.create(HashAlgorithm.MURMUR3), noOfEntries, fixedKeySize, fixedValueSize); + @Test + public void testReadFromAllChunks() { + // Each segment can have only 255 chunks. + noOfEntries = 255; + + segment = new SegmentWithMemoryPool<>(builder().memoryPoolChunkSize(fixedSlotSize)); + List added = addEntriesToSegment(noOfEntries); + Assert.assertEquals(255, added.size()); + for (Record record: added) { + Assert.assertTrue(segment.containsEntry(record.keyBuffer)); + Assert.assertEquals(segment.getEntry(record.keyBuffer), record.entry); + } } @Test - public void testReplace() { + public void testClear() { + // Each segment can have only 255 chunks. + noOfEntries = 255; + + segment = new SegmentWithMemoryPool<>(builder().memoryPoolChunkSize(fixedSlotSize)); + List added = addEntriesToSegment(noOfEntries); + Assert.assertEquals(255, added.size()); + for (Record record: added) { + Assert.assertTrue(segment.containsEntry(record.keyBuffer)); + Assert.assertEquals(segment.getEntry(record.keyBuffer), record.entry); + } + segment.clear(); + for (Record record: added) { + Assert.assertFalse(segment.containsEntry(record.keyBuffer)); + Assert.assertEquals(segment.getEntry(record.keyBuffer), null); + } - int fixedKeySize = 8; - int fixedValueSize = 18; - int noOfEntries = 1000; - int noOfChunks = 10; - int fixedSlotSize = MemoryPoolHashEntries.HEADER_SIZE + fixedKeySize + fixedValueSize; + } - OffHeapHashTableBuilder builder = OffHeapHashTableBuilder - .newBuilder() - .fixedKeySize(fixedKeySize) - .fixedValueSize(fixedValueSize) - // chunkSize set such that noOfEntries/2 can set filled in one. - .memoryPoolChunkSize(noOfEntries/noOfChunks * fixedSlotSize) - .valueSerializer(HashTableTestUtils.byteArraySerializer); + @Test + public void testReplace() { + noOfEntries = 1000; + noOfChunks = 10; - SegmentWithMemoryPool segment = new SegmentWithMemoryPool<>(builder); + segment = new SegmentWithMemoryPool<>(builder()); - Hasher hasher = Hasher.create(HashAlgorithm.MURMUR3); - Map map = new HashMap<>(); - for (int i = 0; i < noOfEntries; i++) { - byte[] key = HashTableTestUtils.randomBytes(fixedKeySize); - KeyBuffer k = new KeyBuffer(key); - k.finish(hasher); - byte[] value = HashTableTestUtils.randomBytes(fixedValueSize); - map.put(k, value); - segment.putEntry(key, value, k.hash(), true, null); - } + List records = addEntriesToSegment(800); // takes 800 slots + List largeRecords = addLargeEntriesToSegment(50); // takes 50 * 4 = 200 slots + int totalEntries = 850; Assert.assertEquals(segment.numberOfChunks(), noOfChunks); - Assert.assertEquals(segment.size(), noOfEntries); - Assert.assertEquals(segment.putAddCount(), noOfEntries); + Assert.assertEquals(segment.size(), totalEntries); + Assert.assertEquals(segment.putAddCount(), totalEntries); Assert.assertEquals(segment.freeListSize(), 0); - map.forEach((k, v) -> { - Assert.assertTrue(segment.putEntry(k.buffer, HashTableTestUtils.randomBytes(fixedValueSize), k.hash(), false, v)); + Stream.concat(records.stream(), largeRecords.stream()).forEach(record -> { + ByteArrayEntry newEntry = serializer.randomEntry(record.keyBuffer.size()); + Assert.assertTrue(segment.putEntry(record.keyBuffer, newEntry, false, record.entry)); }); // we have replaced all values. no new chunks should // have been allocated. Assert.assertEquals(segment.numberOfChunks(), noOfChunks); - Assert.assertEquals(segment.size(), noOfEntries); - Assert.assertEquals(segment.putAddCount(), noOfEntries); + Assert.assertEquals(segment.size(), totalEntries); + Assert.assertEquals(segment.putAddCount(), totalEntries); Assert.assertEquals(segment.freeListSize(), 0); - Assert.assertEquals(segment.putReplaceCount(), noOfEntries); + Assert.assertEquals(segment.putReplaceCount(), totalEntries); // All slots in chunk should be written to. for (int i = 0; i < segment.numberOfChunks(); i++) { @@ -200,57 +361,162 @@ public void testReplace() { } @Test - public void testRehash() { + public void testLongChains() { + noOfEntries = 2000; + noOfChunks = 4; + + segment = new SegmentWithMemoryPool<>(OffHeapHashTableBuilder + .newBuilder(serializer) + .fixedKeySize(fixedKeySize) + .memoryPoolChunkSize(noOfEntries/noOfChunks * fixedSlotSize) + .hashTableSize(256) // small table, for long chains + .loadFactor(10) // so large it won't rehash + ); + + Record testy = createRecord(4); + List small1 = addEntriesToSegment(500); + List large1 = addMaliciousLargeEntriesToSegment(500, testy.keyBuffer.buffer); + List small2 = addEntriesToSegment(500); + List large2 = addLargeEntriesToSegment(500); + + // if this is not null, it means the segment 'search' algorithm did not properly skip over + // the extended key slot when matching keys, as the MaliciousLargeEntries have keys that mimic + // the 'testy' record + Assert.assertNull(segment.getEntry(testy.keyBuffer), "found fake key in extended key slot, should not have"); + Assert.assertEquals(segment.rehashes(), 0); + + // we should have long chains with small single slot keys mixed with the large multi-slot keys + + containsAll(small1); + containsAll(large1); + containsAll(small2); + containsAll(large2); + } - int fixedKeySize = 8; - int fixedValueSize = 18; - int noOfEntries = 100_000; - int noOfChunks = 10; - int fixedSlotSize = MemoryPoolHashEntries.HEADER_SIZE + fixedKeySize + fixedValueSize; + @Test + public void testRehashSmallKeys() { + noOfEntries = 100_000; + noOfChunks = 10; - OffHeapHashTableBuilder builder = OffHeapHashTableBuilder - .newBuilder() + OffHeapHashTableBuilder builder = OffHeapHashTableBuilder + .newBuilder(serializer) .fixedKeySize(fixedKeySize) - .fixedValueSize(fixedValueSize) + // chunkSize set such that noOfEntries/2 can set filled in one. .memoryPoolChunkSize(noOfEntries/noOfChunks * fixedSlotSize) .hashTableSize(noOfEntries/8) // size of table less than number of entries, this will trigger a rehash. - .loadFactor(1) - .valueSerializer(HashTableTestUtils.byteArraySerializer); + .loadFactor(1); - SegmentWithMemoryPool segment = new SegmentWithMemoryPool<>(builder); - Hasher hasher = Hasher.create(HashAlgorithm.MURMUR3); - List records = addEntriesToSegment(segment, hasher, noOfEntries, fixedKeySize, fixedValueSize); + segment = new SegmentWithMemoryPool<>(builder); + List records = addEntriesToSegment(noOfEntries); Assert.assertEquals(segment.size(), noOfEntries); Assert.assertEquals(segment.rehashes(), 3); Assert.assertEquals(segment.putAddCount(), noOfEntries); - records.forEach(r -> Assert.assertEquals(segment.getEntry(r.keyBuffer), r.value)); + containsAll(records); + } + + @Test + public void testRehashMixedKeys() { + noOfEntries = 100_000; + noOfChunks = 10; + + int numSmallKeys = 60_000; + int numLargeKeys = 10_000; // long keys take 4 slots, so 100_000 total slots used + int totalKeys = numSmallKeys + numLargeKeys; // 70_000 + + OffHeapHashTableBuilder builder = OffHeapHashTableBuilder + .newBuilder(serializer) + .fixedKeySize(fixedKeySize) + // chunkSize set such that noOfEntries/2 slots fill one chunk. + .memoryPoolChunkSize(noOfEntries/noOfChunks * fixedSlotSize) + .hashTableSize(totalKeys/10) // size of table less than number of entries, this will trigger multiple rehashes. + .loadFactor(1.25f); // load factor ensures many chains of keys per hash table slot + + segment = new SegmentWithMemoryPool<>(builder); + // rehash will only trigger with large keys present if added first, so all rehash cycles will contain large keys + // and most will contain a mix + List largeRecords = addLargeEntriesToSegment(numLargeKeys); + List records = addEntriesToSegment(numSmallKeys); + + Assert.assertEquals(segment.size(), totalKeys); + Assert.assertEquals(segment.numberOfSlots(), noOfEntries); + Assert.assertEquals(segment.rehashes(), 3); // (8192 * 1.1) ~9000 -> ~18000 -> ~36000 -> ~72000 + Assert.assertEquals(segment.putAddCount(), totalKeys); + + containsAll(records); + containsAll(largeRecords); + } + + private void containsAll(List records) { + records.forEach(r -> Assert.assertEquals(segment.getEntry(r.keyBuffer), r.entry)); } + private List addEntriesToSegment(int noOfEntries) { + List records = new ArrayList<>(); + for (int i = 0; i < noOfEntries; i++) { + Record rec = createRecord(fixedKeySize - (i & 0x1)); // 50% are one byte smaller, so not all slots are full + records.add(rec); + segment.putEntry(rec.keyBuffer, rec.entry, true, null); + } + return records; + } - private List addEntriesToSegment(SegmentWithMemoryPool segment, Hasher hasher, int noOfEntries, int fixedKeySize, int fixedValueSize) { + // generate entries that require four slots per key + private List addLargeEntriesToSegment(int noOfEntries) { List records = new ArrayList<>(); for (int i = 0; i < noOfEntries; i++) { - byte[] key = HashTableTestUtils.randomBytes(fixedKeySize); - KeyBuffer k = new KeyBuffer(key); - k.finish(hasher); - byte[] value = HashTableTestUtils.randomBytes(fixedValueSize); - records.add(new Record(k, value)); - segment.putEntry(key, value, k.hash(), true, null); + Record rec = createRecord((4 *fixedKeySize) + (3 * serializer.entrySize()) - (i & 1)); + records.add(rec); + segment.putEntry(rec.keyBuffer, rec.entry, true, null); } + return records; + } + // generate entries for large keys where the extended blocks appear to be valid ordinary blocks with the given fake key. + private List addMaliciousLargeEntriesToSegment(int noOfEntries, byte[] fakeKey) { + List records = new ArrayList<>(); + for (int i = 0; i < noOfEntries; i++) { + Record rec = createMaliciousRecord(fakeKey, (4 *fixedKeySize) + (3 * serializer.entrySize()) - (i & 1)); + records.add(rec); + segment.putEntry(rec.keyBuffer, rec.entry, true, null); + } return records; } + private Record createRecord(byte[] key) { + KeyBuffer k = new KeyBuffer(key); + k.finish(hasher); + ByteArrayEntry entry = serializer.randomEntry(key.length); + return new Record(k, entry); + } + + private Record createRecord(int keySize) { + byte[] key = HashTableTestUtils.randomBytes(keySize); + return createRecord(key); + } + + private Record createMaliciousRecord(byte[] fakeKey, int keySize) { + byte[] key = HashTableTestUtils.randomBytes(keySize); + long address = Uns.allocate(5); + try { + HashEntry.serializeSizes(address, (short)fakeKey.length, 16); + Uns.copyMemory(address, 0, key, fixedKeySize, 5); + } finally { + Uns.free(address); + } + System.arraycopy(fakeKey, 0, key, fixedKeySize + 5, fakeKey.length); + return createRecord(key); + } + private static class Record { final KeyBuffer keyBuffer; - final byte[] value; + final ByteArrayEntry entry; - public Record(KeyBuffer keyBuffer, byte[] value) { + public Record(KeyBuffer keyBuffer, ByteArrayEntry entry) { this.keyBuffer = keyBuffer; - this.value = value; + this.entry = entry; } } } diff --git a/src/test/java/com/oath/halodb/SequenceNumberTest.java b/src/test/java/com/oath/halodb/SequenceNumberTest.java index 711581a..e32fa68 100644 --- a/src/test/java/com/oath/halodb/SequenceNumberTest.java +++ b/src/test/java/com/oath/halodb/SequenceNumberTest.java @@ -5,9 +5,6 @@ package com.oath.halodb; -import org.testng.Assert; -import org.testng.annotations.Test; - import java.io.File; import java.util.ArrayList; import java.util.Arrays; @@ -15,6 +12,9 @@ import java.util.List; import java.util.stream.Collectors; +import org.testng.Assert; +import org.testng.annotations.Test; + /** * @author Pulkit Goel */ @@ -38,7 +38,7 @@ public void testSequenceNumber(HaloDBOptions options) throws Exception { List sequenceNumbers = new ArrayList<>(); int count = 1; while (haloDBFileIterator.hasNext()) { - Record record = haloDBFileIterator.next(); + RecordEntry record = haloDBFileIterator.next(); sequenceNumbers.add(record.getSequenceNumber()); Assert.assertEquals(record.getSequenceNumber(), count++); } @@ -47,11 +47,11 @@ public void testSequenceNumber(HaloDBOptions options) throws Exception { // open and read the content again HaloDB reopenedDb = getTestDBWithoutDeletingFiles(directory, options); - List records = new ArrayList<>(); - reopenedDb.newIterator().forEachRemaining(records::add); + List records = new ArrayList<>(); + reopenedDb.newIterator().forEachRemaining(record -> records.add((RecordIterated) record)); // Verify that the sequence number is still present after reopening the DB - sequenceNumbers = records.stream().map(record -> record.getRecordMetaData().getSequenceNumber()).collect(Collectors.toList()); + sequenceNumbers = records.stream().map(record -> record.getSequenceNumber()).collect(Collectors.toList()); count = 1; for (long s : sequenceNumbers) { Assert.assertEquals(s, count++); @@ -68,7 +68,7 @@ public void testSequenceNumber(HaloDBOptions options) throws Exception { sequenceNumbers = new ArrayList<>(); count = 110; while (haloDBFileIterator.hasNext()) { - Record record = haloDBFileIterator.next(); + RecordEntry record = haloDBFileIterator.next(); sequenceNumbers.add(record.getSequenceNumber()); Assert.assertEquals(record.getSequenceNumber(), count++); } @@ -109,7 +109,7 @@ public void testSequenceNumber(HaloDBOptions options) throws Exception { sequenceNumbers = new ArrayList<>(); count = 229; while (haloDBFileIterator.hasNext()) { - Record record = haloDBFileIterator.next(); + RecordEntry record = haloDBFileIterator.next(); sequenceNumbers.add(record.getSequenceNumber()); Assert.assertEquals(record.getSequenceNumber(), count++); } diff --git a/src/test/java/com/oath/halodb/SyncWriteTest.java b/src/test/java/com/oath/halodb/SyncWriteTest.java index 9e95e03..c425dc3 100644 --- a/src/test/java/com/oath/halodb/SyncWriteTest.java +++ b/src/test/java/com/oath/halodb/SyncWriteTest.java @@ -1,12 +1,12 @@ package com.oath.halodb; -import org.testng.Assert; -import org.testng.annotations.Test; - import java.io.IOException; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; +import org.testng.Assert; +import org.testng.annotations.Test; + import mockit.Invocation; import mockit.Mock; import mockit.MockUp; @@ -62,15 +62,15 @@ public void flushToDisk(Invocation invocation) throws IOException { }; HaloDBOptions options = new HaloDBOptions(); - // value set to make sure that flush to disk will be called once. + // value set to make sure that flush to disk will be called once. options.setFlushDataSizeBytes(10 * 1024 - 1); HaloDB db = getTestDB(directory, options); int noOfRecords = 10; - TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024 - Record.Header.HEADER_SIZE); + TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024 - RecordEntry.Header.HEADER_SIZE); // 10 records of size 1024 each was inserted and flush size was set to 10 * 1024 - 1, - // therefore data will be flushed to disk once. + // therefore data will be flushed to disk once. Assert.assertEquals(dataFileCount.get(), 1); } @@ -95,12 +95,12 @@ public void flushToDisk(Invocation invocation) throws IOException { }; HaloDBOptions options = new HaloDBOptions(); - // value set to make sure that flush to disk will not be called. + // value set to make sure that flush to disk will not be called. options.setFlushDataSizeBytes(1024 * 1024 * 1024); HaloDB db = getTestDB(directory, options); int noOfRecords = 100; - List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024 - Record.Header.HEADER_SIZE); + List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024 - RecordEntry.Header.HEADER_SIZE); for (Record r : records) { db.delete(r.getKey()); } diff --git a/src/test/java/com/oath/halodb/TestUtils.java b/src/test/java/com/oath/halodb/TestUtils.java index 3ade02c..1f75cae 100644 --- a/src/test/java/com/oath/halodb/TestUtils.java +++ b/src/test/java/com/oath/halodb/TestUtils.java @@ -5,9 +5,6 @@ package com.oath.halodb; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; @@ -28,6 +25,9 @@ import java.util.Set; import java.util.stream.Collectors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + public class TestUtils { private static final Logger logger = LoggerFactory.getLogger(TestUtils.class); @@ -51,7 +51,7 @@ static List insertRandomRecordsOfSize(HaloDB db, int noOfRecords, int si for (int i = 0; i < noOfRecords; i++) { byte[] key; if (size > 0) { - key = TestUtils.generateRandomByteArray(random.nextInt(Math.min(Byte.MAX_VALUE-1, size))+1); + key = TestUtils.generateRandomByteArray(random.nextInt(Math.min(Byte.MAX_VALUE-1, size))+1); } else { key = TestUtils.generateRandomByteArray(); @@ -118,7 +118,7 @@ static List updateRecordsWithSize(HaloDB db, List records, int s records.forEach(record -> { try { - byte[] value = TestUtils.generateRandomByteArray(size-record.getKey().length-Record.Header.HEADER_SIZE); + byte[] value = TestUtils.generateRandomByteArray(size-record.getKey().length-RecordEntry.Header.HEADER_SIZE); db.put(record.getKey(), value); updated.add(new Record(record.getKey(), value)); } catch (HaloDBException e) { diff --git a/src/test/java/com/oath/halodb/TombstoneFileCleanUpTest.java b/src/test/java/com/oath/halodb/TombstoneFileCleanUpTest.java index 7e26761..3903184 100644 --- a/src/test/java/com/oath/halodb/TombstoneFileCleanUpTest.java +++ b/src/test/java/com/oath/halodb/TombstoneFileCleanUpTest.java @@ -1,13 +1,17 @@ package com.oath.halodb; -import org.testng.Assert; -import org.testng.annotations.Test; - import java.io.File; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; import java.util.stream.Collectors; +import org.testng.Assert; +import org.testng.annotations.Test; + public class TombstoneFileCleanUpTest extends TestBase { @Test(dataProvider = "Options") @@ -23,14 +27,14 @@ public void testDeleteAllRecords(HaloDBOptions options) throws HaloDBException, int noOfRecordsPerFile = 1024; int noOfFiles = 100; int noOfRecords = noOfRecordsPerFile * noOfFiles; - List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-Record.Header.HEADER_SIZE); + List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-RecordEntry.Header.HEADER_SIZE); // delete all records for (Record r : records) { db.delete(r.getKey()); } - // all files will be deleted except for the last one as it is the current write file. + // all files will be deleted except for the last one as it is the current write file. TestUtils.waitForCompactionToComplete(db); // close and open the db. @@ -45,7 +49,7 @@ public void testDeleteAllRecords(HaloDBOptions options) throws HaloDBException, tombstoneFile.open(); TombstoneFile.TombstoneFileIterator iterator = tombstoneFile.newIterator(); - //Make sure that only 1024 tombstones from the last data file are left in the tombstone file after clean up. + //Make sure that only 1024 tombstones from the last data file are left in the tombstone file after clean up. int tombstoneCount = 0; List remaining = records.stream().skip(noOfRecords - noOfRecordsPerFile).collect(Collectors.toList()); while (iterator.hasNext()) { @@ -108,28 +112,28 @@ public void testDeleteRecordsWithoutCompaction(HaloDBOptions options) throws IOE int noOfRecordsPerFile = 1024; int noOfFiles = 100; int noOfRecords = noOfRecordsPerFile * noOfFiles; - List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-Record.Header.HEADER_SIZE); + List records = TestUtils.insertRandomRecordsOfSize(db, noOfRecords, 1024-RecordEntry.Header.HEADER_SIZE); // delete first record from each file, since compaction threshold is 1 none of the files will be compacted. for (int i = 0; i < noOfRecords; i+=noOfRecordsPerFile) { db.delete(records.get(i).getKey()); } - // get the tombstone file. + // get the tombstone file. File[] originalTombstoneFiles = FileUtils.listTombstoneFiles(new File(directory)); Assert.assertEquals(originalTombstoneFiles.length, 1); TestUtils.waitForCompactionToComplete(db); - // close and open db. + // close and open db. db.close(); db = getTestDBWithoutDeletingFiles(directory, options); - // make sure that the old tombstone file was deleted. + // make sure that the old tombstone file was deleted. Assert.assertFalse(originalTombstoneFiles[0].exists()); // Since none of the files were compacted we cannot delete any of the tombstone records - // as the stale version of records still exist in the db. + // as the stale version of records still exist in the db. // find the new tombstone file and make sure that all the tombstone records were copied. File[] tombstoneFilesAfterOpen = FileUtils.listTombstoneFiles(new File(directory)); @@ -184,7 +188,7 @@ public void testWithCleanUpTurnedOff(HaloDBOptions options) throws IOException, db.close(); db = getTestDBWithoutDeletingFiles(directory, options); - // clean up was disabled; tombstone file should be the same. + // clean up was disabled; tombstone file should be the same. File[] tombstoneFilesAfterOpen = FileUtils.listTombstoneFiles(new File(directory)); Assert.assertEquals(tombstoneFilesAfterOpen.length, 1); Assert.assertEquals(tombstoneFilesAfterOpen[0].getName(), originalTombstoneFiles[0].getName()); @@ -196,7 +200,7 @@ public void testWithCleanUpTurnedOff(HaloDBOptions options) throws IOException, @Test(dataProvider = "Options") public void testCopyMultipleTombstoneFiles(HaloDBOptions options) throws HaloDBException, IOException { - //Test to make sure that rollover to tombstone files work correctly during cleanup. + //Test to make sure that rollover to tombstone files work correctly during cleanup. String directory = TestUtils.getTestDirectory("TombstoneFileCleanUpTest", "testCopyMultipleTombstoneFiles"); diff --git a/src/test/java/com/oath/halodb/TombstoneFileTest.java b/src/test/java/com/oath/halodb/TombstoneFileTest.java index 0c8fd5a..bf0563f 100644 --- a/src/test/java/com/oath/halodb/TombstoneFileTest.java +++ b/src/test/java/com/oath/halodb/TombstoneFileTest.java @@ -5,11 +5,6 @@ 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; @@ -20,13 +15,18 @@ import java.util.ArrayList; import java.util.List; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + public class TombstoneFileTest { - private File directory = new File(TestUtils.getTestDirectory("TombstoneFileTest")); + private final File directory = new File(TestUtils.getTestDirectory("TombstoneFileTest")); private DBDirectory dbDirectory; private TombstoneFile file; private File backingFile; - private int fileId = 100; + private final int fileId = 100; private FileTime createdTime; @BeforeMethod @@ -56,9 +56,9 @@ public void testRepairFileWithCorruptedEntry() throws IOException { int noOfRecords = 1000; List records = insertTestRecords(noOfRecords); - // add a corrupted entry to the file. + // add a corrupted entry to the file. int sequenceNumber = noOfRecords + 100; - TombstoneEntry corrupted = new TombstoneEntry(TestUtils.generateRandomByteArray(), sequenceNumber, -1, 21); + TombstoneEntry corrupted = new TombstoneEntry(TestUtils.generateRandomByteArray(), sequenceNumber, -1, (byte)2); try(FileChannel channel = FileChannel.open( Paths.get(directory.getCanonicalPath(), fileId + TombstoneFile.TOMBSTONE_FILE_NAME).toAbsolutePath(), StandardOpenOption.APPEND)) { ByteBuffer[] data = corrupted.serialize(); @@ -80,7 +80,7 @@ public void testRepairFileWithCorruptedKeySize() throws IOException { // add a corrupted entry to the file. int sequenceNumber = noOfRecords + 100; - TombstoneEntry corrupted = new TombstoneEntry(TestUtils.generateRandomByteArray(), sequenceNumber, -1, 13); + TombstoneEntry corrupted = new TombstoneEntry(TestUtils.generateRandomByteArray(), sequenceNumber, -1, (byte)13); try(FileChannel channel = FileChannel.open( Paths.get(directory.getCanonicalPath(), fileId + TombstoneFile.TOMBSTONE_FILE_NAME).toAbsolutePath(), StandardOpenOption.APPEND)) { ByteBuffer[] data = corrupted.serialize(); @@ -102,7 +102,7 @@ public void testRepairFileWithIncompleteEntry() throws IOException { // add a corrupted entry to the file. int sequenceNumber = noOfRecords + 100; - TombstoneEntry corrupted = new TombstoneEntry(TestUtils.generateRandomByteArray(), sequenceNumber, -1, 17); + TombstoneEntry corrupted = new TombstoneEntry(TestUtils.generateRandomByteArray(), sequenceNumber, -1, (byte)17); try(FileChannel channel = FileChannel.open( Paths.get(directory.getCanonicalPath(), fileId + TombstoneFile.TOMBSTONE_FILE_NAME).toAbsolutePath(), StandardOpenOption.APPEND)) { ByteBuffer[] data = corrupted.serialize(); @@ -137,7 +137,7 @@ private void verifyData(TombstoneFile file, List records) throws private List insertTestRecords(int number) throws IOException { List records = new ArrayList<>(); for (int i = 0; i < number; i++) { - TombstoneEntry e = new TombstoneEntry(TestUtils.generateRandomByteArray(), i, -1, 1); + TombstoneEntry e = new TombstoneEntry(TestUtils.generateRandomByteArray(), i, -1, (byte)1); file.write(e); records.add(new TombstoneEntry(e.getKey(), e.getSequenceNumber(), e.computeCheckSum(), e.getVersion())); }