diff --git a/.gitignore b/.gitignore index e81a89f7..abf98337 100644 --- a/.gitignore +++ b/.gitignore @@ -2,3 +2,4 @@ build/ .idea/ *.iml +logs/ diff --git a/.travis.yml b/.travis.yml index 1c30205c..e2fbe74c 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,7 +1,7 @@ language: java sudo: false jdk: - - oraclejdk9 + - oraclejdk11 script: ./gradlew clean build after_success: - bash <(curl -s https://codecov.io/bash) diff --git a/README.md b/README.md index e1a507d7..1bd7ddb4 100644 --- a/README.md +++ b/README.md @@ -53,5 +53,26 @@ To benchmark Uppend: ```sh ./gradlew clean fatJar -java -jar build/libs/uppend-all-*.jar benchmark --help +java -jar build/libs/uppend-all-*.jar --help ``` + +To run tests without verbose output +```sh + ./gradlew test -i +``` + +To run tests in a specific path +```sh + ./gradlew test --tests com.upserve.uppend.blobs* +``` + +Example script to fork the benchmark with a system resource monitor like IOSTAT +```sh +trap "kill 0" EXIT + +java -jar build/libs/uppend-all-0.0.2-91-g6abbf45.dirty.jar benchmark ../foo/test & BENCHMARK_PID=$! +iostat -d 2 & IOSTAT_PID=$! + +wait $BENCHMARK_PID +kill $IOSTAT_PID +``` \ No newline at end of file diff --git a/build.gradle b/build.gradle index 64627946..7eb5feaf 100644 --- a/build.gradle +++ b/build.gradle @@ -13,6 +13,8 @@ plugins { } import org.apache.tools.ant.filters.ExpandProperties +import org.gradle.api.tasks.testing.logging.TestExceptionFormat +import org.gradle.api.tasks.testing.logging.TestLogEvent if (new File('.git').exists() && (exec { commandLine "sh", "-c", "git --version" @@ -30,10 +32,6 @@ group 'com.upserve' description = """Uppend: fast, append-only key-multivalue store""" -task wrapper(type: Wrapper) { - gradleVersion = '4.4' -} - // TODO unused-dependency is broken - claims all dependencies are unused! //gradleLint.rules += 'unused-dependency' @@ -42,6 +40,10 @@ apply plugin: 'maven' apply plugin: 'jacoco' apply plugin: 'com.bmuschko.nexus' +jacoco { + toolVersion = "0.8.2" // Fixed to resolve issue with JDK 11 in Gradle 4.X.Y +} + sourceCompatibility = 1.9 targetCompatibility = 1.9 // Requires 1.9 or greater due to unsafe memory access in MappebByteBuffer - Oracle Incident Report 9119653 @@ -52,8 +54,7 @@ repositories { dependencies { compile 'com.google.guava:guava:21.0' - compile 'com.github.ben-manes.caffeine:caffeine:2.6.2' - compile 'info.picocli:picocli:3.0.0' + compile 'info.picocli:picocli:4.0.1' compile 'io.dropwizard.metrics:metrics-core:3.2.3' compile 'it.unimi.dsi:fastutil:7.0.13' // compile 'me.lemire.integercompression:JavaFastPFOR:0.1.11' @@ -62,12 +63,6 @@ dependencies { testCompile 'junit:junit:4.12' testCompile 'org.apache.logging.log4j:log4j-api:2.8' testCompile 'org.mockito:mockito-core:2.18.3' - - gradleLint.ignore { - // gradle-lint doesn't like log4j because it's not used directly - compile 'org.apache.logging.log4j:log4j-core:2.8' - compile 'org.apache.logging.log4j:log4j-slf4j-impl:2.8' - } } tasks.withType(JavaCompile) { @@ -93,6 +88,10 @@ processResources { } task fatJar(type: Jar) { + dependencies { + compile 'org.apache.logging.log4j:log4j-core:2.8' + compile 'org.apache.logging.log4j:log4j-slf4j-impl:2.8' + } manifest { attributes 'Main-Class': 'com.upserve.uppend.Uppend' } @@ -101,6 +100,44 @@ task fatJar(type: Jar) { with jar } +tasks.withType(Test) { + // From https://stackoverflow.com/a/36130467/2136991 + testLogging { + // set options for log level LIFECYCLE + events TestLogEvent.FAILED, + TestLogEvent.PASSED, + TestLogEvent.SKIPPED, + TestLogEvent.STANDARD_OUT + exceptionFormat TestExceptionFormat.FULL + showExceptions true + showCauses true + showStackTraces true + + // set options for log level DEBUG and INFO + debug { + + events TestLogEvent.STARTED, + TestLogEvent.FAILED, + TestLogEvent.PASSED, + TestLogEvent.SKIPPED, + TestLogEvent.STANDARD_ERROR, + TestLogEvent.STANDARD_OUT + exceptionFormat TestExceptionFormat.FULL + } + info.events = debug.events + info.exceptionFormat = debug.exceptionFormat + + afterSuite { desc, result -> + if (!desc.parent) { // will match the outermost suite + def output = "Results: ${result.resultType} (${result.testCount} tests, ${result.successfulTestCount} successes, ${result.failedTestCount} failures, ${result.skippedTestCount} skipped)" + def startItem = '| ', endItem = ' |' + def repeatLength = startItem.length() + output.length() + endItem.length() + println('\n' + ('-' * repeatLength) + '\n' + startItem + output + endItem + '\n' + ('-' * repeatLength)) + } + } + } +} + jacocoTestReport { reports { xml.enabled = true diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index c6952461..e2ea6fd6 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -3,4 +3,4 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-4.7-all.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-4.10.2-all.zip diff --git a/jitpack.yml b/jitpack.yml index 57d91684..9fb64a8d 100644 --- a/jitpack.yml +++ b/jitpack.yml @@ -1,3 +1,2 @@ jdk: - - oraclejdk9 - \ No newline at end of file + - oraclejdk11 diff --git a/src/main/java/com/upserve/uppend/AppendOnlyStore.java b/src/main/java/com/upserve/uppend/AppendOnlyStore.java index 4c28907d..22143c30 100644 --- a/src/main/java/com/upserve/uppend/AppendOnlyStore.java +++ b/src/main/java/com/upserve/uppend/AppendOnlyStore.java @@ -1,10 +1,5 @@ package com.upserve.uppend; -import com.github.benmanes.caffeine.cache.stats.CacheStats; -import com.upserve.uppend.lookup.FlushStats; - -import java.io.Flushable; - /** * Add byte arrays under a key and partition, and retrieve them. Note the * expectation that the byte arrays are appended to the value, which is an @@ -46,6 +41,4 @@ public interface AppendOnlyStore extends ReadOnlyAppendOnlyStore, RegisteredFlus * @return the name */ String getName(); - - FlushStats getFlushStats(); } \ No newline at end of file diff --git a/src/main/java/com/upserve/uppend/AppendOnlyStoreBuilder.java b/src/main/java/com/upserve/uppend/AppendOnlyStoreBuilder.java index 891428b2..2e7a78ee 100644 --- a/src/main/java/com/upserve/uppend/AppendOnlyStoreBuilder.java +++ b/src/main/java/com/upserve/uppend/AppendOnlyStoreBuilder.java @@ -1,12 +1,8 @@ package com.upserve.uppend; -import com.upserve.uppend.blobs.PageCache; import com.upserve.uppend.metrics.AppendOnlyStoreWithMetrics; -import java.util.concurrent.*; - public class AppendOnlyStoreBuilder extends FileStoreBuilder { - // Blocked Longs Config Options public static final int DEFAULT_BLOBS_PER_BLOCK = 127; @@ -14,14 +10,8 @@ public class AppendOnlyStoreBuilder extends FileStoreBuilder val > 0) + .summaryStatistics(); + return new PartitionStats(metadataBlobFile.getAllocatedPageCount(), + longKeyFile.getAllocatedPageCount(), + blobFile.getAllocatedPageCount(), + Arrays.stream(lookups).mapToLong(LookupData::getMetadataLookupMissCount).sum(), + Arrays.stream(lookups).mapToLong(LookupData::getMetadataLookupHitCount).sum(), + metadataStats.getSum(), + Arrays.stream(lookups).mapToLong(LookupData::getFindKeyTimer).sum(), + Arrays.stream(lookups).mapToLong(LookupData::getFlushedKeyCount).sum(), + Arrays.stream(lookups).mapToLong(LookupData::getFlushCount).sum(), + metadataStats.getCount(), + metadataStats.getMax() + ); + } + + private AppendStorePartition(VirtualPageFile longKeyFile, VirtualPageFile metadataBlobFile, VirtualPageFile blobsFile, BlockedLongs blocks, int hashCount, int flushThreshold, int reloadInterval, boolean readOnly) { + super(longKeyFile, metadataBlobFile, hashCount, flushThreshold, reloadInterval, readOnly); this.blocks = blocks; this.blobFile = blobsFile; - blobs = IntStream.range(0, hashSize) + blobs = IntStream.range(0, hashCount) .mapToObj(virtualFileNumber -> new VirtualAppendOnlyBlobStore(virtualFileNumber, blobsFile)) .toArray(VirtualAppendOnlyBlobStore[]::new); } @@ -90,8 +99,10 @@ Stream read(String key) { LookupKey lookupKey = new LookupKey(key); final int hash = keyHash(lookupKey); - return blocks.values(lookups[hash].getValue(lookupKey)).parallel().mapToObj(blobs[hash]::read); - // Consider sorting by blob pos or even grouping by the page of the blob pos and then flat-mapping the reads by page. + // Values stream can now be parallel, but it breaks everything... + LongStream longStream = blocks.values(lookups[hash].getValue(lookupKey)); + return longStream.mapToObj(blobs[hash]::read); + //return blocks.lazyValues(lookups[hash].getValue(lookupKey)).parallel().mapToObj(blobs[hash]::read); } Stream readSequential(String key) { @@ -109,7 +120,7 @@ byte[] readLast(String key) { } Stream>> scan() { - return IntStream.range(0, hashSize) + return IntStream.range(0, hashCount) .parallel() .boxed() .flatMap(virtualFileNumber -> @@ -120,9 +131,8 @@ Stream>> scan() { ); } - void scan(BiConsumer> callback) { - IntStream.range(0, hashSize) + IntStream.range(0, hashCount) .parallel() .boxed() .forEach(virtualFileNumber -> @@ -131,26 +141,12 @@ void scan(BiConsumer> callback) { } Stream keys() { - return IntStream.range(0, hashSize) + return IntStream.range(0, hashCount) .parallel() .boxed() .flatMap(virtualFileNumber -> lookups[virtualFileNumber].keys().map(LookupKey::string)); } - @Override - public void flush() throws IOException { - log.debug("Starting flush for partition: {}", lookupCache.getPartition()); - - Arrays.stream(lookups).parallel().forEach(LookupData::flush); - - longKeyFile.flush(); - metadataBlobFile.flush(); - blobFile.flush(); - blocks.flush(); - - log.debug("Finished flush for partition: {}", lookupCache.getPartition()); - } - BlockStats blockedLongStats() { return blocks.stats(); } @@ -166,10 +162,8 @@ void clear() throws IOException { @Override public void close() throws IOException { - flush(); + super.close(); - longKeyFile.close(); - metadataBlobFile.close(); blobFile.close(); blocks.close(); } diff --git a/src/main/java/com/upserve/uppend/AutoFlusher.java b/src/main/java/com/upserve/uppend/AutoFlusher.java index 9029aaf8..f1c09ca3 100644 --- a/src/main/java/com/upserve/uppend/AutoFlusher.java +++ b/src/main/java/com/upserve/uppend/AutoFlusher.java @@ -13,16 +13,11 @@ public class AutoFlusher { private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private static final int FLUSH_EXEC_POOL_NUM_THREADS = 20; - private static final ConcurrentMap flushableDelays = new ConcurrentHashMap<>(); private static final ConcurrentMap> delayFlushables = new ConcurrentHashMap<>(); private static final ConcurrentMap delayFutures = new ConcurrentHashMap<>(); - private static final ConcurrentLinkedQueue flushTasks = new ConcurrentLinkedQueue<>(); - private static final ThreadFactory threadFactory; - public static final ExecutorService flushExecPool; public static final ForkJoinPool flusherWorkPool; @@ -40,10 +35,6 @@ public class AutoFlusher { return t; }; - AtomicInteger flushExecPoolThreadNumber = new AtomicInteger(); - ThreadFactory flushExecPoolThreadFactory = r -> new Thread(threadGroup, r, "auto-flush-exec-pool-" + flushExecPoolThreadNumber.incrementAndGet()); - flushExecPool = Executors.newFixedThreadPool(FLUSH_EXEC_POOL_NUM_THREADS, flushExecPoolThreadFactory); - threadFactoryFunction = name -> pool -> { final ForkJoinWorkerThread worker = ForkJoinPool.defaultForkJoinWorkerThreadFactory.newThread(pool); @@ -51,9 +42,8 @@ public class AutoFlusher { return worker; }; - forkJoinPoolFunction = name -> new ForkJoinPool( - Runtime.getRuntime().availableProcessors(), + Runtime.getRuntime().availableProcessors() * 2, threadFactoryFunction.apply(name), (t, e) -> { log.error("In pool {}, thread {} threw exception {}", name, t, e); @@ -62,7 +52,6 @@ public class AutoFlusher { ); flusherWorkPool = forkJoinPoolFunction.apply("flush-worker"); - } public static synchronized void register(int delaySeconds, Flushable flushable) { @@ -138,9 +127,6 @@ private static void flush(int delaySeconds) { } public static void submitWork(Runnable runnable) { - - ForkJoinTask task = flusherWorkPool.submit(runnable); + flusherWorkPool.submit(runnable); } - - } diff --git a/src/main/java/com/upserve/uppend/BlockStats.java b/src/main/java/com/upserve/uppend/BlockStats.java index e5ba1eb4..46433d0c 100644 --- a/src/main/java/com/upserve/uppend/BlockStats.java +++ b/src/main/java/com/upserve/uppend/BlockStats.java @@ -10,9 +10,7 @@ public class BlockStats { private final long allocCount; private final long valuesReadCount; - public static BlockStats ZERO_STATS = new BlockStats(0,0,0,0,0); - public BlockStats(int pagesLoaded, long size, long appendCount, long allocCount, long valuesReadCount) { this.pagesLoaded = pagesLoaded; this.size = size; @@ -21,8 +19,6 @@ public BlockStats(int pagesLoaded, long size, long appendCount, long allocCount, this.valuesReadCount = valuesReadCount; } - - public int getPagesLoaded() { return pagesLoaded; } @@ -50,6 +46,7 @@ public String toString() { ", size=" + size + ", appendCount=" + appendCount + ", allocCount=" + allocCount + + ", valuesReadCount=" + valuesReadCount + '}'; } @@ -65,7 +62,7 @@ public BlockStats minus(BlockStats other) { } public BlockStats add(BlockStats other) { - if (Objects.isNull(other)) throw new NullPointerException("BlockStats minus method argument is null"); + if (Objects.isNull(other)) throw new NullPointerException("BlockStats add method argument is null"); return new BlockStats( pagesLoaded + other.pagesLoaded, size + other.size, diff --git a/src/main/java/com/upserve/uppend/BlockedLongs.java b/src/main/java/com/upserve/uppend/BlockedLongs.java index b5034d6d..cb99b406 100644 --- a/src/main/java/com/upserve/uppend/BlockedLongs.java +++ b/src/main/java/com/upserve/uppend/BlockedLongs.java @@ -1,7 +1,6 @@ package com.upserve.uppend; import com.google.common.util.concurrent.Striped; -import com.upserve.uppend.util.*; import org.slf4j.Logger; import java.io.*; @@ -12,7 +11,6 @@ import java.util.*; import java.util.concurrent.atomic.*; import java.util.concurrent.locks.Lock; -import java.util.function.Supplier; import java.util.stream.*; public class BlockedLongs implements AutoCloseable, Flushable { @@ -22,8 +20,8 @@ public class BlockedLongs implements AutoCloseable, Flushable { private static final int LOCK_SIZE = 10007; private final Striped stripedLocks; - private static final int PAGE_SIZE = 4 * 1024 * 1024; // allocate 4 MB chunks - private static final int MAX_PAGES = 1024 * 1024; // max 4 TB (~800 MB heap) + private static final int PAGE_SIZE = 128 * 1024 * 1024; // allocate 128 MB chunks + private static final int MAX_PAGES = 32 * 1024; // max 4 TB private final Path file; @@ -33,8 +31,6 @@ public class BlockedLongs implements AutoCloseable, Flushable { private final FileChannel blocks; private final MappedByteBuffer[] pages; - private final Supplier bufferLocal; - private final FileChannel blocksPos; private final MappedByteBuffer posBuf; private final AtomicLong posMem; @@ -46,7 +42,6 @@ public class BlockedLongs implements AutoCloseable, Flushable { private final LongAdder allocCounter; private final LongAdder valuesReadCounter; - public BlockedLongs(Path file, int valuesPerBlock, boolean readOnly) { if (file == null) { throw new IllegalArgumentException("null file"); @@ -75,8 +70,6 @@ public BlockedLongs(Path file, int valuesPerBlock, boolean readOnly) { this.valuesPerBlock = valuesPerBlock; blockSize = 16 + valuesPerBlock * 8; - // size | -next - // prev | -last StandardOpenOption[] openOptions; if (readOnly) { openOptions = new StandardOpenOption[]{StandardOpenOption.READ}; @@ -92,19 +85,14 @@ public BlockedLongs(Path file, int valuesPerBlock, boolean readOnly) { if (readOnly) { stripedLocks = null; - pages = null; - currentPage = null; - } else { stripedLocks = Striped.lock(LOCK_SIZE); - - pages = new MappedByteBuffer[MAX_PAGES]; - - ensurePage(0); - currentPage = new AtomicInteger(0); } - bufferLocal = ThreadLocalByteBuffers.threadLocalByteBufferSupplier(blockSize); + pages = new MappedByteBuffer[MAX_PAGES]; + + ensurePage(0); + currentPage = new AtomicInteger(0); try { blocksPos = FileChannel.open(posFile, openOptions); @@ -223,59 +211,110 @@ public LongStream values(Long pos) { return LongStream.empty(); } + long[] longs = valuesArray(pos); + return Arrays.stream(longs); + } + + public long[] valuesArray(Long pos) { + /* + This method is deprecated. It will be replaced with a Spliterator that is block aware to allow parallel reads + */ + if (pos < 0 || pos > size()) { log.error("Bad position value {} in file {} of size {}", pos, file, size()); - return LongStream.empty(); - } - - ByteBuffer buf = readBlock(pos); - if (buf == null) { - return LongStream.empty(); + return new long[]{}; } - buf.flip(); // size | -next // prev | -last - - long size = buf.getLong(); - buf.getLong(); + final long size = readLong(pos); if (size < 0) { long nextPos = -size; long[] values = new long[valuesPerBlock]; for (int i = 0; i < valuesPerBlock; i++) { - values[i] = buf.getLong(); + values[i] = readLong(pos + 16 + i * 8); + } + + long[] additionalValues = valuesArray(nextPos); + + if (additionalValues.length == 0){ + return values; + } else { + long[] result = new long[valuesPerBlock + additionalValues.length]; + System.arraycopy(values,0,result,0, valuesPerBlock); + System.arraycopy(additionalValues,0,result,valuesPerBlock, additionalValues.length); + return result; } - return LongStreams.lazyConcat(Arrays.stream(values), () -> values(nextPos)); } else if (size > valuesPerBlock) { throw new IllegalStateException("too high num values: expected <= " + valuesPerBlock + ", got " + size); } else if (size == 0) { - return LongStream.empty(); + return new long[]{}; } else { int numValues = (int) size; long[] values = new long[numValues]; for (int i = 0; i < numValues; i++) { - values[i] = buf.getLong(); + values[i] = readLong(pos + 16 + i * 8); } - if (log.isTraceEnabled()) { - String valuesStr = Arrays.toString(values); - log.trace("got values from {} at {}: {}", file, pos, valuesStr); - } - return Arrays.stream(values); + return values; } } +// public LongStream lazyValues(Long pos) { +// /* +// Lazy values is much slower in Performance tests with a large number of blocks. +// This method is retained for reference purposes only till the Spliterator is impelemnted. +// */ +// log.trace("streaming values from {} at {}", file, pos); +// +// valuesReadCounter.increment(); +// +// if (pos == null) { +// // pos will be null for missing keys +// return LongStream.empty(); +// } +// +// if (pos < 0 || pos > size()) { +// log.error("Bad position value {} in file {} of size {}", pos, file, size()); +// return LongStream.empty(); +// } +// +// // size | -next +// // prev | -last +// final long size = readLong(pos); +// +// if (size < 0) { +// long nextPos = -size; +// long[] values = new long[valuesPerBlock]; +// for (int i = 0; i < valuesPerBlock; i++) { +// values[i] = readLong(pos + 16 + i * 8); +// } +// return LongStreams.lazyConcat(Arrays.stream(values), () -> values(nextPos)); +// } else if (size > valuesPerBlock) { +// throw new IllegalStateException("too high num values: expected <= " + valuesPerBlock + ", got " + size); +// } else if (size == 0) { +// return LongStream.empty(); +// } else { +// +// int numValues = (int) size; +// long[] values = new long[numValues]; +// for (int i = 0; i < numValues; i++) { +// values[i] = readLong(pos + 16 + i * 8); +// } +// if (log.isTraceEnabled()) { +// String valuesStr = Arrays.toString(values); +// log.trace("got values from {} at {}: {}", file, pos, valuesStr); +// } +// return Arrays.stream(values); +// } +// } + public long lastValue(long pos) { log.trace("reading last value from {} at {}", file, pos); if (pos >= posMem.get()) { return -1; } - ByteBuffer buf = readBlock(pos); - if (buf == null) { - return -1; - } - buf.flip(); // size | -next // prev | -last @@ -368,22 +407,6 @@ public void trim() { } } - private ByteBuffer readBlock(long pos) { - ByteBuffer buf = bufferLocal.get(); - try { - int numRead = blocks.read(buf, pos); - if (numRead == -1) { - return null; - } - if (numRead != blockSize) { - throw new RuntimeException("read bad block size from " + file + " at pos " + pos + ": got " + numRead + ", expected " + blockSize); - } - } catch (Exception e) { - throw new RuntimeException("unable to read block at pos " + pos + ": " + file, e); - } - return buf; - } - private long readLong(long pos) { int pagePos = (int) (pos % (long) PAGE_SIZE); return page(pos).getLong(pagePos); @@ -424,7 +447,8 @@ private MappedByteBuffer ensurePage(int pageIndex) { if (page == null) { long pageStart = (long) pageIndex * PAGE_SIZE; try { - page = blocks.map(FileChannel.MapMode.READ_WRITE, pageStart, PAGE_SIZE); + FileChannel.MapMode mapMode = readOnly ? FileChannel.MapMode.READ_ONLY : FileChannel.MapMode.READ_WRITE; + page = blocks.map(mapMode, pageStart, PAGE_SIZE); } catch (IOException e) { throw new UncheckedIOException("unable to map page at page index " + pageIndex + " (" + pageStart + " + " + PAGE_SIZE + ") in " + file, e); } diff --git a/src/main/java/com/upserve/uppend/CounterStore.java b/src/main/java/com/upserve/uppend/CounterStore.java index e4a3da80..e83ef87c 100644 --- a/src/main/java/com/upserve/uppend/CounterStore.java +++ b/src/main/java/com/upserve/uppend/CounterStore.java @@ -1,9 +1,5 @@ package com.upserve.uppend; -import com.upserve.uppend.lookup.FlushStats; - -import java.io.Flushable; - /** * Keep counters for partitioned keys. */ @@ -69,6 +65,4 @@ default long increment(String partitionEntropy, String key) { * @return the name of the datastore for reporting purposes */ String getName(); - - FlushStats getFlushStats(); } diff --git a/src/main/java/com/upserve/uppend/CounterStoreBuilder.java b/src/main/java/com/upserve/uppend/CounterStoreBuilder.java index 4d271660..aa802be2 100644 --- a/src/main/java/com/upserve/uppend/CounterStoreBuilder.java +++ b/src/main/java/com/upserve/uppend/CounterStoreBuilder.java @@ -3,13 +3,12 @@ import com.upserve.uppend.metrics.CounterStoreWithMetrics; public class CounterStoreBuilder extends FileStoreBuilder { - public CounterStore build() { return build(false); } public CounterStore build(boolean readOnly) { - if (readOnly && flushDelaySeconds != DEFAULT_FLUSH_DELAY_SECONDS) + if (readOnly && getFlushDelaySeconds() != DEFAULT_FLUSH_DELAY_SECONDS) throw new IllegalStateException("Can not set flush delay seconds in read only mode"); CounterStore store = new FileCounterStore(readOnly, this); if (isStoreMetrics()) store = new CounterStoreWithMetrics(store, getStoreMetricsRegistry(), getMetricsRootName()); @@ -20,47 +19,8 @@ public ReadOnlyCounterStore buildReadOnly() { return build(true); } - private static CounterStoreBuilder defaultTestBuilder = new CounterStoreBuilder() - .withStoreName("test") - .withInitialLookupKeyCacheSize(64) - .withMaximumLookupKeyCacheWeight(100 * 1024) - .withInitialMetaDataCacheSize(64) - .withMaximumMetaDataCacheWeight(100 * 1024) - .withMetaDataPageSize(1024) - .withLongLookupHashSize(16) - .withLookupPageSize(16 * 1024) - .withCacheMetrics(); - - public static CounterStoreBuilder getDefaultTestBuilder() { - return defaultTestBuilder; - } - @Override public String toString() { - return "CounterStoreBuilder{" + - "storeName='" + storeName + '\'' + - ", partitionSize=" + partitionSize + - ", lookupHashSize=" + lookupHashSize + - ", lookupPageSize=" + lookupPageSize + - ", initialLookupPageCacheSize=" + initialLookupPageCacheSize + - ", maximumLookupPageCacheSize=" + maximumLookupPageCacheSize + - ", maximumLookupKeyCacheWeight=" + maximumLookupKeyCacheWeight + - ", initialLookupKeyCacheSize=" + initialLookupKeyCacheSize + - ", maximumMetaDataCacheWeight=" + maximumMetaDataCacheWeight + - ", initialMetaDataCacheSize=" + initialMetaDataCacheSize + - ", metadataTTL=" + metadataTTL + - ", metaDataPageSize=" + metaDataPageSize + - ", lookupKeyCacheExecutorService=" + lookupKeyCacheExecutorService + - ", lookupMetaDataCacheExecutorService=" + lookupMetaDataCacheExecutorService + - ", lookupPageCacheExecutorService=" + lookupPageCacheExecutorService + - ", flushDelaySeconds=" + flushDelaySeconds + - ", flushThreshold=" + flushThreshold + - ", dir=" + dir + - ", storeMetricsRegistry=" + storeMetricsRegistry + - ", metricsRootName='" + metricsRootName + '\'' + - ", storeMetrics=" + storeMetrics + - ", cacheMetricsRegistry=" + cacheMetricsRegistry + - ", cacheMetrics=" + cacheMetrics + - '}'; + return "CounterStoreBuilder{}" + super.toString(); } } diff --git a/src/main/java/com/upserve/uppend/CounterStorePartition.java b/src/main/java/com/upserve/uppend/CounterStorePartition.java index 6ba2a79e..f7214306 100644 --- a/src/main/java/com/upserve/uppend/CounterStorePartition.java +++ b/src/main/java/com/upserve/uppend/CounterStorePartition.java @@ -13,40 +13,32 @@ import java.util.function.ObjLongConsumer; import java.util.stream.*; -public class CounterStorePartition extends Partition implements Flushable, Closeable { +public class CounterStorePartition extends Partition { private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - public static CounterStorePartition createPartition(Path partentDir, String partition, int hashSize, int flushThreshold, int metadataPageSize, PageCache keyPageCache, LookupCache lookupCache) { - validatePartition(partition); - Path partitiondDir = partentDir.resolve(partition); - try { - Files.createDirectories(partitiondDir); - } catch (IOException e) { - throw new UncheckedIOException("Unable to make partition directory: " + partitiondDir, e); - } + public static CounterStorePartition createPartition(Path parentDir, String partition, int hashCount, int targetBufferSize, int flushThreshold, int reloadInterval, int metadataPageSize, int keyPageSize) { + Path partitionDir = validatePartition(parentDir, partition); - VirtualPageFile metadata = new VirtualPageFile(metadataPath(partitiondDir), hashSize, metadataPageSize, false); - VirtualPageFile keys = new VirtualPageFile(keysPath(partitiondDir), hashSize, false, keyPageCache); + VirtualPageFile metadata = new VirtualPageFile(metadataPath(partitionDir), hashCount, metadataPageSize, adjustedTargetBufferSize(metadataPageSize, hashCount, targetBufferSize), false); + VirtualPageFile keys = new VirtualPageFile(keysPath(partitionDir), hashCount, keyPageSize, adjustedTargetBufferSize(keyPageSize, hashCount, targetBufferSize), false); - - return new CounterStorePartition(keys, metadata, PartitionLookupCache.create(partition, lookupCache), hashSize, flushThreshold, false); + return new CounterStorePartition(keys, metadata, hashCount, flushThreshold, reloadInterval, false); } - public static CounterStorePartition openPartition(Path partentDir, String partition, int hashSize, int flushThreshold, int metadataPageSize, PageCache keyPageCache, LookupCache lookupCache, boolean readOnly) { + public static CounterStorePartition openPartition(Path partentDir, String partition, int hashCount, int targetBufferSize, int flushThreshold, int reloadInterval, int metadataPageSize, int keyPageSize, boolean readOnly) { validatePartition(partition); Path partitiondDir = partentDir.resolve(partition); if (!(Files.exists(metadataPath(partitiondDir)) && Files.exists(keysPath(partitiondDir)))) return null; + VirtualPageFile metadata = new VirtualPageFile(metadataPath(partitiondDir), hashCount, metadataPageSize, adjustedTargetBufferSize(metadataPageSize, hashCount, targetBufferSize), readOnly); + VirtualPageFile keys = new VirtualPageFile(keysPath(partitiondDir), hashCount, keyPageSize, targetBufferSize, readOnly); - VirtualPageFile metadata = new VirtualPageFile(metadataPath(partitiondDir), hashSize, metadataPageSize, readOnly); - VirtualPageFile keys = new VirtualPageFile(keysPath(partitiondDir), hashSize, readOnly, keyPageCache); - - return new CounterStorePartition(keys, metadata, PartitionLookupCache.create(partition, lookupCache), hashSize, flushThreshold, false); + return new CounterStorePartition(keys, metadata, hashCount, flushThreshold, reloadInterval, false); } - private CounterStorePartition(VirtualPageFile longKeyFile, VirtualPageFile metadataBlobFile, PartitionLookupCache lookupCache, int hashSize, int flushThreshold, boolean readOnly) { - super(longKeyFile, metadataBlobFile, lookupCache, hashSize, flushThreshold, readOnly); + private CounterStorePartition(VirtualPageFile longKeyFile, VirtualPageFile metadataBlobFile, int hashCount, int flushThreshold, int reloadInterval, boolean readOnly) { + super(longKeyFile, metadataBlobFile, hashCount, flushThreshold, reloadInterval, readOnly); } public Long set(String key, long value) { @@ -71,7 +63,7 @@ public Long get(String key) { } public Stream> scan() { - return IntStream.range(0, hashSize) + return IntStream.range(0, hashCount) .parallel() .boxed() .flatMap(virtualFileNumber -> lookups[virtualFileNumber].scan().map(entry -> Maps.immutableEntry(entry.getKey().string(), entry.getValue()))); @@ -79,41 +71,22 @@ public Stream> scan() { public void scan(ObjLongConsumer callback) { - IntStream.range(0, hashSize) + IntStream.range(0, hashCount) .parallel() .boxed() .forEach(virtualFileNumber -> lookups[virtualFileNumber].scan((keyLookup, value) -> callback.accept(keyLookup.string(), value))); } Stream keys() { - return IntStream.range(0, hashSize) + return IntStream.range(0, hashCount) .parallel() .boxed() .flatMap(virtualFileNumber -> lookups[virtualFileNumber].keys().map(LookupKey::string)); } - @Override - public void flush() throws IOException { - log.debug("Starting flush for partition: {}", lookupCache.getPartition()); - - Arrays.stream(lookups).parallel().forEach(LookupData::flush); - - longKeyFile.flush(); - metadataBlobFile.flush(); - log.debug("Finished flush for partition: {}", lookupCache.getPartition()); - } - void clear() throws IOException { longKeyFile.close(); metadataBlobFile.close(); SafeDeleting.removeDirectory(longKeyFile.getFilePath().getParent()); } - - @Override - public void close() throws IOException { - flush(); - - longKeyFile.close(); - metadataBlobFile.close(); - } } diff --git a/src/main/java/com/upserve/uppend/FileAppendOnlyStore.java b/src/main/java/com/upserve/uppend/FileAppendOnlyStore.java index 611df0f9..cba30293 100644 --- a/src/main/java/com/upserve/uppend/FileAppendOnlyStore.java +++ b/src/main/java/com/upserve/uppend/FileAppendOnlyStore.java @@ -1,42 +1,24 @@ package com.upserve.uppend; -import com.github.benmanes.caffeine.cache.stats.CacheStats; -import com.upserve.uppend.blobs.PageCache; -import com.upserve.uppend.lookup.*; -import com.upserve.uppend.util.SafeDeleting; import org.slf4j.Logger; -import java.io.*; import java.lang.invoke.MethodHandles; -import java.nio.channels.ClosedChannelException; import java.util.*; import java.util.function.*; import java.util.stream.Stream; -import static com.upserve.uppend.BlockStats.ZERO_STATS; - public class FileAppendOnlyStore extends FileStore implements AppendOnlyStore { private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private final PageCache blobPageCache; - private final PageCache keyPageCache; - private final LookupCache lookupCache; - private final Function openPartitionFunction; private final Function createPartitionFunction; FileAppendOnlyStore(boolean readOnly, AppendOnlyStoreBuilder builder) { - super(builder.getDir(), builder.getFlushDelaySeconds(), builder.getPartitionSize(), readOnly, builder.getStoreName()); - - blobPageCache = builder.buildBlobPageCache(getName()); - - keyPageCache = builder.buildLookupPageCache(getName()); - - lookupCache = builder.buildLookupCache(getName(), readOnly); + super(builder.getDir(), builder.getFlushDelaySeconds(), builder.getPartitionCount(), readOnly, builder.getStoreName()); - openPartitionFunction = partitionKey -> AppendStorePartition.openPartition(partitionsDir, partitionKey, builder.getLookupHashSize(), builder.getFlushThreshold(), builder.getMetadataPageSize(), builder.getBlobsPerBlock(), blobPageCache, keyPageCache, lookupCache, readOnly); + openPartitionFunction = partitionKey -> AppendStorePartition.openPartition(partitionsDir, partitionKey, builder.getLookupHashCount(), builder.getTargetBufferSize(), builder.getFlushThreshold(), builder.getMetadataTTL(), builder.getMetadataPageSize(), builder.getBlobsPerBlock(), builder.getBlobPageSize(), builder.getLookupPageSize(), readOnly); - createPartitionFunction = partitionKey -> AppendStorePartition.createPartition(partitionsDir, partitionKey, builder.getLookupHashSize(), builder.getFlushThreshold(), builder.getMetadataPageSize(), builder.getBlobsPerBlock(), blobPageCache, keyPageCache, lookupCache); + createPartitionFunction = partitionKey -> AppendStorePartition.createPartition(partitionsDir, partitionKey, builder.getLookupHashCount(), builder.getTargetBufferSize(), builder.getFlushThreshold(), builder.getMetadataTTL(), builder.getMetadataPageSize(), builder.getBlobsPerBlock(), builder.getBlobPageSize(), builder.getLookupPageSize()); } @Override @@ -45,33 +27,13 @@ public String getName() { } @Override - public FlushStats getFlushStats() { - return lookupCache.getFlushStats(); - } - - @Override - public CacheStats getBlobPageCacheStats() { - return blobPageCache.stats(); - } - - @Override - public CacheStats getKeyPageCacheStats() { - return keyPageCache.stats(); - } - - @Override - public CacheStats getLookupKeyCacheStats() { - return lookupCache.keyStats(); - } - - @Override - public CacheStats getMetadataCacheStats() { - return lookupCache.metadataStats(); + public BlockStats getBlockLongStats() { + return partitionMap.values().parallelStream().map(AppendStorePartition::blockedLongStats).reduce(BlockStats.ZERO_STATS, BlockStats::add); } @Override - public BlockStats getBlockLongStats() { - return partitionMap.values().parallelStream().map(AppendStorePartition::blockedLongStats).reduce(ZERO_STATS, BlockStats::add); + public PartitionStats getPartitionStats(){ + return partitionMap.values().parallelStream().map(AppendStorePartition::getPartitionStats).reduce(PartitionStats.ZERO_STATS, PartitionStats::add); } @Override @@ -131,20 +93,6 @@ public void scan(BiConsumer> callback) { .forEach(partitionObject -> partitionObject.scan(callback)); } - @Override - public void clear() { - if (readOnly) throw new RuntimeException("Can not clear a store opened in read only mode:" + name); - log.trace("clearing"); - - closeInternal(); - - try { - SafeDeleting.removeDirectory(partitionsDir); - } catch (IOException e) { - throw new UncheckedIOException("Failed to clear partitions directory", e); - } - } - @Override Function getOpenPartitionFunction() { return openPartitionFunction; @@ -154,51 +102,4 @@ Function getOpenPartitionFunction() { Function getCreatePartitionFunction() { return createPartitionFunction; } - - @Override - protected void flushInternal() { - // Flush lookups, then blocks, then blobs, since this is the access order of a read. - // Check non null because the super class is registered in the autoflusher before the constructor finishes - if (readOnly) throw new RuntimeException("Can not flush a store opened in read only mode:" + name); - - partitionMap.values().parallelStream().forEach(appendStorePartition -> { - try { - appendStorePartition.flush(); - } catch (ClosedChannelException e) { - if (isClosed.get()) { - log.debug("Tried to flush a closed store {}", name, e); - } else { - throw new UncheckedIOException("Error flushing store " + name, e); - } - - } catch (IOException e) { - if (isClosed.get()) - throw new UncheckedIOException("Error flushing store " + name, e); - } - }); - } - - @Override - public void trimInternal() { - if (!readOnly) flushInternal(); - lookupCache.flush(); - blobPageCache.flush(); - keyPageCache.flush(); - } - - @Override - protected void closeInternal() { - partitionMap.values().parallelStream().forEach(appendStorePartition -> { - try { - appendStorePartition.close(); - } catch (IOException e) { - throw new UncheckedIOException("Error closing store " + name, e); - } - }); - - partitionMap.clear(); - lookupCache.flush(); - blobPageCache.flush(); - keyPageCache.flush(); - } } diff --git a/src/main/java/com/upserve/uppend/FileCounterStore.java b/src/main/java/com/upserve/uppend/FileCounterStore.java index c9bb42a2..d32bf9e7 100644 --- a/src/main/java/com/upserve/uppend/FileCounterStore.java +++ b/src/main/java/com/upserve/uppend/FileCounterStore.java @@ -1,12 +1,7 @@ package com.upserve.uppend; -import com.github.benmanes.caffeine.cache.stats.CacheStats; -import com.upserve.uppend.blobs.PageCache; -import com.upserve.uppend.lookup.*; -import com.upserve.uppend.util.SafeDeleting; import org.slf4j.Logger; -import java.io.*; import java.lang.invoke.MethodHandles; import java.util.*; import java.util.function.*; @@ -15,19 +10,14 @@ public class FileCounterStore extends FileStore implements CounterStore { private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private final PageCache keyPageCache; - private final LookupCache lookupCache; private final Function openPartitionFunction; private final Function createPartitionFunction; FileCounterStore(boolean readOnly, CounterStoreBuilder builder) { - super(builder.getDir(), builder.getFlushDelaySeconds(), builder.getPartitionSize(), readOnly, builder.getStoreName()); + super(builder.getDir(), builder.getFlushDelaySeconds(), builder.getPartitionCount(), readOnly, builder.getStoreName()); - keyPageCache = builder.buildLookupPageCache(getName()); - lookupCache = builder.buildLookupCache(getName(), readOnly); - - openPartitionFunction = partitionKey -> CounterStorePartition.openPartition(partitionsDir, partitionKey, builder.getLookupHashSize(), builder.getFlushThreshold(), builder.getMetadataPageSize(), keyPageCache, lookupCache, readOnly); - createPartitionFunction = partitionKey -> CounterStorePartition.createPartition(partitionsDir, partitionKey, builder.getLookupHashSize(), builder.getFlushThreshold(), builder.getMetadataPageSize(), keyPageCache, lookupCache); + openPartitionFunction = partitionKey -> CounterStorePartition.openPartition(partitionsDir, partitionKey, builder.getLookupHashCount(), builder.getTargetBufferSize(), builder.getFlushThreshold(), builder.getMetadataTTL(), builder.getMetadataPageSize(), builder.getLookupPageSize(), readOnly); + createPartitionFunction = partitionKey -> CounterStorePartition.createPartition(partitionsDir, partitionKey, builder.getLookupHashCount(), builder.getTargetBufferSize(), builder.getFlushThreshold(), builder.getMetadataTTL(), builder.getMetadataPageSize(), builder.getLookupPageSize()); } @Override @@ -75,25 +65,6 @@ public void scan(ObjLongConsumer callback) { .forEach(partitionObject -> partitionObject.scan(callback)); } - @Override - public FlushStats getFlushStats() { - return lookupCache.getFlushStats(); - } - @Override - public CacheStats getKeyPageCacheStats() { - return keyPageCache.stats(); - } - - @Override - public CacheStats getLookupKeyCacheStats() { - return lookupCache.keyStats(); - } - - @Override - public CacheStats getMetadataCacheStats() { - return lookupCache.metadataStats(); - } - @Override public long keyCount() { return streamPartitions() @@ -101,38 +72,6 @@ public long keyCount() { .sum(); } - @Override - public void clear() { - log.trace("clearing"); - if (readOnly) throw new RuntimeException("Can not clear a store opened in read only mode:" + dir); - - log.trace("clearing"); - - partitionMap.values().stream().forEach(counterStorePartition -> { - try { - counterStorePartition.clear(); - } catch (IOException e) { - throw new UncheckedIOException("Failed to clear counter store partition", e); - } - }); - - try { - SafeDeleting.removeDirectory(partitionsDir); - } catch (IOException e) { - throw new UncheckedIOException("Failed to clear partitions directory", e); - } - partitionMap.clear(); - lookupCache.flush(); - keyPageCache.flush(); - } - - @Override - public void trimInternal() { - if (!readOnly) flushInternal(); - lookupCache.flush(); - keyPageCache.flush(); - } - @Override Function getOpenPartitionFunction() { return openPartitionFunction; @@ -143,30 +82,4 @@ Function getCreatePartitionFunction() { return createPartitionFunction; } - @Override - protected void flushInternal() { - if (readOnly) throw new RuntimeException("Can not flush a store opened in read only mode:" + dir); - - partitionMap.values().parallelStream().forEach(counterStorePartition -> { - try { - counterStorePartition.flush(); - } catch (IOException e) { - throw new UncheckedIOException("Error flushing store " + dir, e); - } - }); - } - - @Override - protected void closeInternal() { - partitionMap.values().parallelStream().forEach(counterStorePartition -> { - try { - counterStorePartition.close(); - } catch (IOException e) { - throw new UncheckedIOException("Error closing store " + dir, e); - } - }); - - lookupCache.flush(); - keyPageCache.flush(); - } } diff --git a/src/main/java/com/upserve/uppend/FileStore.java b/src/main/java/com/upserve/uppend/FileStore.java index ec76b1a5..8bb455b6 100644 --- a/src/main/java/com/upserve/uppend/FileStore.java +++ b/src/main/java/com/upserve/uppend/FileStore.java @@ -1,6 +1,7 @@ package com.upserve.uppend; import com.google.common.hash.*; +import com.upserve.uppend.util.SafeDeleting; import org.slf4j.Logger; import java.io.*; @@ -9,36 +10,36 @@ import java.nio.charset.StandardCharsets; import java.nio.file.*; import java.util.*; -import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; import java.util.stream.Stream; -abstract class FileStore implements AutoCloseable, RegisteredFlushable, Trimmable { - public static final int MAX_NUM_PARTITIONS = 9999; +abstract class FileStore implements AutoCloseable, RegisteredFlushable, Trimmable { + static final int MAX_NUM_PARTITIONS = 9999; private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - protected final Path dir; - protected final Path partitionsDir; + final Path dir; + final Path partitionsDir; private final int flushDelaySeconds; - protected final Map partitionMap; + final ConcurrentHashMap partitionMap; protected final boolean readOnly; protected final String name; private final Path lockPath; private final FileChannel lockChan; private final FileLock lock; - private final int partitionSize; + private final int partitionCount; private final boolean doHashPartitionValues; - protected final AtomicBoolean isClosed; + final AtomicBoolean isClosed; private static final int PARTITION_HASH_SEED = 626433832; private final HashFunction hashFunction = Hashing.murmur3_32(PARTITION_HASH_SEED); - FileStore(Path dir, int flushDelaySeconds, int partitionSize, boolean readOnly, String name) { + FileStore(Path dir, int flushDelaySeconds, int partitionCount, boolean readOnly, String name) { if (dir == null) { throw new NullPointerException("null dir"); } @@ -49,18 +50,18 @@ abstract class FileStore implements AutoCloseable, RegisteredFlushable, Trimm throw new UncheckedIOException("unable to mkdirs: " + dir, e); } partitionsDir = dir.resolve("partitions"); - if (partitionSize > MAX_NUM_PARTITIONS) { - throw new IllegalArgumentException("bad partition size: greater than max (" + MAX_NUM_PARTITIONS + "): " + partitionSize); + if (partitionCount > MAX_NUM_PARTITIONS) { + throw new IllegalArgumentException("bad partition count: greater than max (" + MAX_NUM_PARTITIONS + "): " + partitionCount); } - if (partitionSize < 0) { - throw new IllegalArgumentException("bad partition size: negative: " + partitionSize); + if (partitionCount < 0) { + throw new IllegalArgumentException("bad partition count: negative: " + partitionCount); } - this.partitionSize = partitionSize; - if (partitionSize == 0) { + this.partitionCount = partitionCount; + if (partitionCount == 0) { partitionMap = new ConcurrentHashMap<>(); doHashPartitionValues = false; } else { - partitionMap = new ConcurrentHashMap<>(partitionSize); + partitionMap = new ConcurrentHashMap<>(partitionCount); doHashPartitionValues = true; } this.name = name; @@ -83,10 +84,10 @@ abstract class FileStore implements AutoCloseable, RegisteredFlushable, Trimm isClosed = new AtomicBoolean(false); } - protected String partitionHash(String partition) { + String partitionHash(String partition) { if (doHashPartitionValues) { HashCode hcode = hashFunction.hashBytes(partition.getBytes(StandardCharsets.UTF_8)); - return String.format("%04d", Math.abs(hcode.asInt()) % partitionSize); + return String.format("%04d", Math.abs(hcode.asInt()) % partitionCount); } else { return partition; } @@ -130,22 +131,46 @@ Stream streamPartitions() { return partitionMap.values().parallelStream(); } + @Override + public void flush() { + // Flush lookups, then blocks, then blobs, since this is the access order of a read. + // NPE may occur because the super class is registered in the autoflusher before the constructor finishes + if (readOnly) throw new RuntimeException("Can not flush a store opened in read only mode:" + dir); - protected abstract void flushInternal() throws IOException; + log.debug("Flushing!"); - protected abstract void closeInternal() throws IOException; + ForkJoinTask task = AutoFlusher.flusherWorkPool.submit(() -> + partitionMap.values().parallelStream().forEach(T::flush) + ); + try { + task.get(); + } catch (InterruptedException e) { + log.error("Flush interrupted", e); - protected abstract void trimInternal() throws IOException; + } catch (ExecutionException e) { + throw new RuntimeException("Partition map flush failed with", e); + } + + log.debug("Flushed!"); + } @Override - public void trim() { - log.debug("Triming {}", name); + public void trim(){ + log.debug("Trimming!"); + + ForkJoinTask task = AutoFlusher.flusherWorkPool.submit(() -> + partitionMap.values().parallelStream().forEach(T::trim) + ); try { - trimInternal(); - } catch (Exception e) { - log.error("unable to trim {}", name, e); + task.get(); + } catch (InterruptedException e) { + log.error("Trim interrupted", e); + + } catch (ExecutionException e) { + throw new RuntimeException("Partition map trim failed with", e); } - log.debug("Trimed {}", name); + + log.debug("Trimmed!"); } @Override @@ -158,15 +183,17 @@ public void deregister() { AutoFlusher.deregister(this); } - @Override - public void flush() { - log.info("flushing {}", name); + public void clear() { + if (readOnly) throw new RuntimeException("Can not clear a store opened in read only mode:" + name); + log.trace("clearing"); + + closePartitions(); + try { - flushInternal(); - } catch (Exception e) { - log.error("unable to flush {}", name, e); + SafeDeleting.removeDirectory(partitionsDir); + } catch (IOException e) { + throw new UncheckedIOException("Failed to clear partitions directory", e); } - log.info("flushed {}", name); } @Override @@ -178,11 +205,7 @@ public void close() { if (!readOnly && flushDelaySeconds > 0) AutoFlusher.deregister(this); - try { - closeInternal(); - } catch (Exception e) { - log.error("unable to close {}", name, e); - } + closePartitions(); try { lock.release(); @@ -195,4 +218,26 @@ public void close() { log.error("unable to close lock file: " + lockPath, e); } } + + private void closePartitions(){ + ForkJoinTask task = AutoFlusher.flusherWorkPool.submit(() -> + partitionMap.values().parallelStream().forEach(partition -> { + try { + partition.close(); + } catch (IOException e) { + throw new UncheckedIOException("Error closing store " + name, e); + } + }) + ); + + try { + task.get(); + } catch (InterruptedException e) { + log.error("Close interrupted", e); + + } catch (ExecutionException e) { + throw new RuntimeException("Partition map close failed with", e); + } + partitionMap.clear(); + } } diff --git a/src/main/java/com/upserve/uppend/FileStoreBuilder.java b/src/main/java/com/upserve/uppend/FileStoreBuilder.java index 4ee0e757..4177cdf1 100644 --- a/src/main/java/com/upserve/uppend/FileStoreBuilder.java +++ b/src/main/java/com/upserve/uppend/FileStoreBuilder.java @@ -1,73 +1,47 @@ package com.upserve.uppend; import com.codahale.metrics.MetricRegistry; -import com.github.benmanes.caffeine.cache.stats.*; -import com.upserve.uppend.blobs.PageCache; -import com.upserve.uppend.lookup.LookupCache; -import com.upserve.uppend.metrics.MetricsStatsCounter; - import java.nio.file.Path; -import java.util.concurrent.*; -import java.util.function.Supplier; public class FileStoreBuilder> { - static final String LOOKUP_PAGE_CACHE_METRICS = "LookupPageCache"; - static final String BLOB_PAGE_CACHE_METRICS = "BlobPageCache"; - static final String LOOKUP_KEY_CACHE_METRICS = "LookupKeyCache"; - static final String METADATA_CACHE_METRICS = "MetadataCache"; - // Long lookup Cache Options - public static final int DEFAULT_PARTITION_SIZE = 0; - public static final int DEFAULT_LOOKUP_HASH_SIZE = 256; + public static final int DEFAULT_PARTITION_COUNT = 0; + public static final int DEFAULT_LOOKUP_HASH_COUNT = 256; public static final int DEFAULT_LOOKUP_PAGE_SIZE = 256 * 1024; - public static final int DEFAULT_INITIAL_LOOKUP_PAGE_CACHE_SIZE = 1024; - public static final int DEFAULT_MAXIMUM_LOOKUP_PAGE_CACHE_SIZE = 16 * 1024; - public static final long DEFAULT_MAXIMUM_LOOKUP_KEY_CACHE_WEIGHT = 1_000_000; - public static final int DEFAULT_INITIAL_LOOKUP_KEY_CACHE_SIZE = 1000; + public static final int TARGET_PRODUCTION_BUFFER_SIZE = Integer.MAX_VALUE; - public static final long DEFAULT_MAXIMUM_METADATA_CACHE_WEIGHT = 1_000_000; - public static final int DEFAULT_INITIAL_METADATA_CACHE_SIZE = 1000; public static final int DEFAULT_METADATA_PAGE_SIZE = 4096; public static final int DEFAULT_METADATA_TTL = 0; // Off by default! - String storeName = ""; - int partitionSize = DEFAULT_PARTITION_SIZE; - int lookupHashSize = DEFAULT_LOOKUP_HASH_SIZE; - - int lookupPageSize = DEFAULT_LOOKUP_PAGE_SIZE; - int initialLookupPageCacheSize = DEFAULT_INITIAL_LOOKUP_PAGE_CACHE_SIZE; - int maximumLookupPageCacheSize = DEFAULT_MAXIMUM_LOOKUP_PAGE_CACHE_SIZE; + private String storeName = ""; + private int partitionCount = DEFAULT_PARTITION_COUNT; + private int lookupHashCount = DEFAULT_LOOKUP_HASH_COUNT; - long maximumLookupKeyCacheWeight = DEFAULT_MAXIMUM_LOOKUP_KEY_CACHE_WEIGHT; - int initialLookupKeyCacheSize = DEFAULT_INITIAL_LOOKUP_KEY_CACHE_SIZE; + private int lookupPageSize = DEFAULT_LOOKUP_PAGE_SIZE; - long maximumMetaDataCacheWeight = DEFAULT_MAXIMUM_METADATA_CACHE_WEIGHT; - int initialMetaDataCacheSize = DEFAULT_INITIAL_METADATA_CACHE_SIZE; - int metadataTTL = DEFAULT_METADATA_TTL; - int metaDataPageSize = DEFAULT_METADATA_PAGE_SIZE; + private int metadataTTL = DEFAULT_METADATA_TTL; + private int metadataPageSize = DEFAULT_METADATA_PAGE_SIZE; - ExecutorService lookupKeyCacheExecutorService = ForkJoinPool.commonPool(); - ExecutorService lookupMetaDataCacheExecutorService = ForkJoinPool.commonPool(); - ExecutorService lookupPageCacheExecutorService = ForkJoinPool.commonPool(); + private int targetBufferSize = TARGET_PRODUCTION_BUFFER_SIZE; // Store Options public static final int DEFAULT_FLUSH_DELAY_SECONDS = 30; public static final int DEFAULT_FLUSH_THRESHOLD = 1000; - int flushDelaySeconds = DEFAULT_FLUSH_DELAY_SECONDS; - int flushThreshold = DEFAULT_FLUSH_THRESHOLD; - Path dir = null; - MetricRegistry storeMetricsRegistry = null; - String metricsRootName = ""; - boolean storeMetrics = false; - MetricRegistry cacheMetricsRegistry = null; - boolean cacheMetrics = false; + private int flushDelaySeconds = DEFAULT_FLUSH_DELAY_SECONDS; + private int flushThreshold = DEFAULT_FLUSH_THRESHOLD; + private Path dir = null; + private MetricRegistry storeMetricsRegistry = null; + private String metricsRootName = ""; + private boolean storeMetrics = false; + private MetricRegistry cacheMetricsRegistry = null; + private boolean cacheMetrics = false; // Long lookup Cache Options @SuppressWarnings("unchecked") - public T withLongLookupHashSize(int longLookupHashSize) { - this.lookupHashSize = longLookupHashSize; + public T withLongLookupHashCount(int longLookupHashCount) { + this.lookupHashCount = longLookupHashCount; return (T) this; } @@ -78,44 +52,8 @@ public T withLookupPageSize(int lookupPageSize) { } @SuppressWarnings("unchecked") - public T withInitialLookupPageCacheSize(int initialLookupPageCacheSize) { - this.initialLookupPageCacheSize = initialLookupPageCacheSize; - return (T) this; - } - - @SuppressWarnings("unchecked") - public T withMaximumLookupPageCacheSize(int maximumLookupPageCacheSize) { - this.maximumLookupPageCacheSize = maximumLookupPageCacheSize; - return (T) this; - } - - @SuppressWarnings("unchecked") - public T withMaximumLookupKeyCacheWeight(long maximumLookupKeyCacheWeight) { - this.maximumLookupKeyCacheWeight = maximumLookupKeyCacheWeight; - return (T) this; - } - - @SuppressWarnings("unchecked") - public T withInitialLookupKeyCacheSize(int initialLookupKeyCacheSize) { - this.initialLookupKeyCacheSize = initialLookupKeyCacheSize; - return (T) this; - } - - @SuppressWarnings("unchecked") - public T withMaximumMetaDataCacheWeight(long maximumMetaDataCacheWeight) { - this.maximumMetaDataCacheWeight = maximumMetaDataCacheWeight; - return (T) this; - } - - @SuppressWarnings("unchecked") - public T withInitialMetaDataCacheSize(int initialMetaDataCacheSize) { - this.initialMetaDataCacheSize = initialMetaDataCacheSize; - return (T) this; - } - - @SuppressWarnings("unchecked") - public T withMetaDataPageSize(int metaDataPageSize) { - this.metaDataPageSize = metaDataPageSize; + public T withMetadataPageSize(int metadataPageSize) { + this.metadataPageSize = metadataPageSize; return (T) this; } @@ -126,20 +64,8 @@ public T withMetadataTTL(int metadataTTL) { } @SuppressWarnings("unchecked") - public T withLookupKeyCacheExecutorService(ExecutorService lookupKeyCacheExecutorService) { - this.lookupKeyCacheExecutorService = lookupKeyCacheExecutorService; - return (T) this; - } - - @SuppressWarnings("unchecked") - public T withLookupMetaDataCacheExecutorService(ExecutorService lookupMetaDataCacheExecutorService) { - this.lookupMetaDataCacheExecutorService = lookupMetaDataCacheExecutorService; - return (T) this; - } - - @SuppressWarnings("unchecked") - public T withLookupPageCacheExecutorService(ExecutorService lookupPageCacheExecutorService) { - this.lookupPageCacheExecutorService = lookupPageCacheExecutorService; + public T withTargetBufferSize(int targetBufferSize) { + this.targetBufferSize = targetBufferSize; return (T) this; } @@ -151,8 +77,8 @@ public T withStoreName(String storeName) { } @SuppressWarnings("unchecked") - public T withPartitionSize(int partitionSize) { - this.partitionSize = partitionSize; + public T withPartitionCount(int partitionCount) { + this.partitionCount = partitionCount; return (T) this; } @@ -187,17 +113,6 @@ public T withStoreMetrics(MetricRegistry metrics) { return (T) this; } - /** - * Apply a MetricsCounter to the Caffeine Caches used by the store - * - * @return the builder - */ - @SuppressWarnings("unchecked") - public T withCacheMetrics() { - this.cacheMetrics = true; - return (T) this; - } - /** * Use a root name for all metrics * @param metricsRootName the root name under which to register metrics from this store @@ -209,65 +124,8 @@ public T withMetricsRootName(String metricsRootName) { return (T) this; } - /** - * Apply a MetricsCounter to the Caffeine Caches used by the store using a CodaHale MetricsRegistry as the counter - * - * @param metrics a CodaHale storeMetrics registry - * @return the builder - */ - @SuppressWarnings("unchecked") - public T withCacheMetrics(MetricRegistry metrics) { - this.cacheMetricsRegistry = metrics; - this.cacheMetrics = true; - return (T) this; - } - - /** - * Return a StatsCounterSupplier for use in the Caffeine builder or null if cacheMetrics is false - * - * @param elements the string elements to use in registering metrics for this cache - * @return the Supplier or null - */ - public Supplier metricsSupplier(String... elements) { - if (!cacheMetrics) return null; - if (cacheMetricsRegistry != null) { - return () -> new MetricsStatsCounter(cacheMetricsRegistry, String.join(".", elements)); - } else { - return ConcurrentStatsCounter::new; - } - } - - public PageCache buildLookupPageCache(String metricsPrefix) { - return new PageCache( - getLookupPageSize(), - getInitialLookupPageCacheSize(), - getMaximumLookupPageCacheSize(), - getLookupPageCacheExecutorService(), - metricsSupplier(metricsPrefix, LOOKUP_PAGE_CACHE_METRICS) - ); - } - - public LookupCache buildLookupCache(String metricsPrefix) { - return buildLookupCache(metricsPrefix, false); - } - - public LookupCache buildLookupCache(String metricsPrefix, boolean readOnly) { - return new LookupCache( - getInitialLookupKeyCacheSize(), - getMaximumLookupKeyCacheWeight(), - getLookupKeyCacheExecutorService(), - metricsSupplier(metricsPrefix, LOOKUP_KEY_CACHE_METRICS), - getInitialMetaDataCacheSize(), - getMaximumMetaDataCacheWeight(), - readOnly ? getMetadataTTL() : 0, - getLookupMetaDataCacheExecutorService(), - metricsSupplier(metricsPrefix, METADATA_CACHE_METRICS) - ); - } - - - public int getLookupHashSize() { - return lookupHashSize; + public int getLookupHashCount() { + return lookupHashCount; } public int getLookupPageSize() { @@ -278,26 +136,6 @@ public boolean isStoreMetrics() { return storeMetrics; } - public int getInitialLookupPageCacheSize() { - return initialLookupPageCacheSize; - } - - public int getMaximumLookupPageCacheSize() { - return maximumLookupPageCacheSize; - } - - public long getMaximumLookupKeyCacheWeight() { - return maximumLookupKeyCacheWeight; - } - - public int getInitialLookupKeyCacheSize() { - return initialLookupKeyCacheSize; - } - - public long getMaximumMetaDataCacheWeight() { - return maximumMetaDataCacheWeight; - } - public int getFlushDelaySeconds() { return flushDelaySeconds; } @@ -314,45 +152,44 @@ public MetricRegistry getStoreMetricsRegistry() { return storeMetricsRegistry; } - public MetricRegistry getCacheMetricsRegistry() { - return cacheMetricsRegistry; - } - - public boolean isCacheMetrics() { - return cacheMetrics; - } - - public int getInitialMetaDataCacheSize() { - return initialMetaDataCacheSize; - } - public int getMetadataPageSize() { - return metaDataPageSize; + return metadataPageSize; } public int getMetadataTTL() { return metadataTTL; } - public ExecutorService getLookupKeyCacheExecutorService() { - return lookupKeyCacheExecutorService; - } - - public ExecutorService getLookupMetaDataCacheExecutorService() { - return lookupMetaDataCacheExecutorService; - } - - public ExecutorService getLookupPageCacheExecutorService() { - return lookupPageCacheExecutorService; + public int getTargetBufferSize() { + return targetBufferSize; } public String getStoreName() { return storeName.isEmpty() ? getDir().getFileName().toString() : storeName; } - public int getPartitionSize(){ return partitionSize; } + public int getPartitionCount(){ return partitionCount; } public String getMetricsRootName(){ return metricsRootName; } -} - + @Override + public String toString() { + return "FileStoreBuilder{" + + "storeName='" + storeName + '\'' + + ", partitionCount=" + partitionCount + + ", lookupHashCount=" + lookupHashCount + + ", lookupPageSize=" + lookupPageSize + + ", metadataTTL=" + metadataTTL + + ", metadataPageSize=" + metadataPageSize + + ", targetBufferSize=" + targetBufferSize + + ", flushDelaySeconds=" + flushDelaySeconds + + ", flushThreshold=" + flushThreshold + + ", dir=" + dir + + ", storeMetricsRegistry=" + storeMetricsRegistry + + ", metricsRootName='" + metricsRootName + '\'' + + ", storeMetrics=" + storeMetrics + + ", cacheMetricsRegistry=" + cacheMetricsRegistry + + ", cacheMetrics=" + cacheMetrics + + '}'; + } +} diff --git a/src/main/java/com/upserve/uppend/Partition.java b/src/main/java/com/upserve/uppend/Partition.java index 4c5c2cee..80c5cdea 100644 --- a/src/main/java/com/upserve/uppend/Partition.java +++ b/src/main/java/com/upserve/uppend/Partition.java @@ -4,64 +4,99 @@ import com.upserve.uppend.blobs.*; import com.upserve.uppend.lookup.*; -import java.nio.file.Path; -import java.util.Arrays; -import java.util.stream.IntStream; +import java.io.*; +import java.nio.file.*; +import java.util.*; +import java.util.function.*; +import java.util.stream.*; -public abstract class Partition { - private static final int MAX_HASH_SIZE = 1 << 24; /* 16,777,216 */ +import static java.lang.Math.min; + +public abstract class Partition implements Flushable, Closeable, Trimmable { + private static final int MAX_HASH_COUNT = 1 << 14; /* 16,384 */ private static final int HASH_SEED = 219370429; final VirtualPageFile longKeyFile; final VirtualPageFile metadataBlobFile; - final PartitionLookupCache lookupCache; - private final HashFunction hashFunction; - - final int hashSize; + protected final boolean readOnly; + final int hashCount; final LookupData[] lookups; - Partition(VirtualPageFile longKeyFile, VirtualPageFile metadataBlobFile, PartitionLookupCache lookupCache, int hashSize, int flushThreshold, boolean readOnly) { + Partition(VirtualPageFile longKeyFile, VirtualPageFile metadataBlobFile, int hashCount, int flushThreshold, int reloadInterval, boolean readOnly) { this.longKeyFile = longKeyFile; this.metadataBlobFile = metadataBlobFile; - this.lookupCache = lookupCache; + this.hashCount = hashCount; + this.readOnly = readOnly; - this.hashSize = hashSize; - - if (hashSize < 1) { - throw new IllegalArgumentException("hashSize must be >= 1"); + if (hashCount < 1) { + throw new IllegalArgumentException("hashCount must be >= 1"); } - if (hashSize > MAX_HASH_SIZE) { - throw new IllegalArgumentException("hashSize must be <= " + MAX_HASH_SIZE); + if (hashCount > MAX_HASH_COUNT) { + throw new IllegalArgumentException("hashCount must be <= " + MAX_HASH_COUNT); } - if (hashSize == 1) { + if (hashCount == 1) { hashFunction = null; } else { hashFunction = Hashing.murmur3_32(HASH_SEED); } - lookups = IntStream.range(0, hashSize) - .mapToObj(virtualFileNumber -> new LookupData( - new VirtualLongBlobStore(virtualFileNumber, longKeyFile), - new VirtualMutableBlobStore(virtualFileNumber, metadataBlobFile), - lookupCache, - flushThreshold, - readOnly - ) - ) + IntFunction constructorFuntion = lookupDataFunction(readOnly, flushThreshold, reloadInterval); + + lookups = IntStream.range(0, hashCount) + .mapToObj(constructorFuntion) .toArray(LookupData[]::new); } + static Path validatePartition(Path parentDir, String partition) { + validatePartition(partition); + Path partitionDir = parentDir.resolve(partition); + try { + Files.createDirectories(partitionDir); + } catch (IOException e) { + throw new UncheckedIOException("Unable to make partition directory: " + partitionDir, e); + } + return partitionDir; + } + + private IntFunction lookupDataFunction(boolean readOnly, int flushThreshold, int relaodInterval) { + if (readOnly) { + return virtualFileNumber -> LookupData.lookupReader( + new VirtualLongBlobStore(virtualFileNumber, longKeyFile), + new VirtualMutableBlobStore(virtualFileNumber, metadataBlobFile), + relaodInterval + ); + } else { + return virtualFileNumber -> LookupData.lookupWriter( + new VirtualLongBlobStore(virtualFileNumber, longKeyFile), + new VirtualMutableBlobStore(virtualFileNumber, metadataBlobFile), + flushThreshold + ); + } + } + + /** + * A function for estimating an efficient buffer size for key and metadata files + * Use the smaller of 2 pages for every hash or the target buffer size + * @param pageSize the size of the page in bytes + * @param hashCount the hash count of the partition + * @param targetBufferSize The configured maximum target size for buffers + * @return the adjusted buffer size to use for metadata or key data + */ + static int adjustedTargetBufferSize(int pageSize, int hashCount, int targetBufferSize) { + return (int) min((long) (pageSize + 16) * hashCount * 2, (long) targetBufferSize); + } + int keyHash(LookupKey key) { if (hashFunction == null){ return 0; } else { - return Math.abs(hashFunction.hashBytes(key.bytes()).asInt()) % hashSize; + return Math.abs(hashFunction.hashBytes(key.bytes()).asInt()) % hashCount; } } @@ -92,6 +127,24 @@ static void validatePartition(String partition) { } } + @Override + public void flush() { + Arrays.stream(lookups).forEach(LookupData::flush); + } + + @Override + public void trim() { + Arrays.stream(lookups).forEach(LookupData::trim); + } + + @Override + public void close() throws IOException { + if (!readOnly) flush(); + + longKeyFile.close(); + metadataBlobFile.close(); + } + private static boolean isValidPartitionCharStart(char c) { return Character.isJavaIdentifierPart(c); } diff --git a/src/main/java/com/upserve/uppend/PartitionStats.java b/src/main/java/com/upserve/uppend/PartitionStats.java new file mode 100644 index 00000000..03d1326f --- /dev/null +++ b/src/main/java/com/upserve/uppend/PartitionStats.java @@ -0,0 +1,144 @@ +package com.upserve.uppend; + +import java.util.Objects; + +public class PartitionStats { + + private final int metadataPageCount; + private final int keyPageCount; + private final int blobPageCount; + + private final long metadataLookupMissCount; + private final long metadataLookupHitCount; + + private final long metadataSize; + private final long findKeyTimer; + + private final long flushedKeyCount; + private final long flushCount; + + private final long lookups; + private final long maxLookupSize; + + public PartitionStats(int metadataPageCount, int keyPageCount, int blobPageCount, long metadataLookupMissCount, long metadataLookupHitCount, long metadataSize, long findKeyTimer, long flushedKeyCount, long flushCount, long lookups, long maxLookupSize) { + this.metadataPageCount = metadataPageCount; + this.keyPageCount = keyPageCount; + this.blobPageCount = blobPageCount; + this.metadataLookupMissCount = metadataLookupMissCount; + this.metadataLookupHitCount = metadataLookupHitCount; + this.metadataSize = metadataSize; + this.findKeyTimer = findKeyTimer; + this.flushedKeyCount = flushedKeyCount; + this.flushCount = flushCount; + this.lookups = lookups; + this.maxLookupSize = maxLookupSize; + } + + public static PartitionStats ZERO_STATS = new PartitionStats(0,0,0,0,0, 0, 0, 0 ,0, 0, 0); + + public int getMetadataPageCount() { + return metadataPageCount; + } + + public int getKeyPageCount() { + return keyPageCount; + } + + public int getBlobPageCount() { + return blobPageCount; + } + + public long getMetadataLookupHitCount() { + return metadataLookupHitCount; + } + + public long getMetadataLookupMissCount() { + return metadataLookupMissCount; + } + + public long getMetadataSize() { + return metadataSize; + } + + public long getFindKeyTimer() { + return findKeyTimer; + } + + public long getFlushedKeyCount() { return flushedKeyCount; } + + public long getFlushCount() { return flushCount; } + + public long getLookupCount() { return lookups; } + + public long getMaxLookupSize() { return maxLookupSize; } + + @Override + public String toString() { + return "PartitionStats{" + + "metadataPageCount=" + metadataPageCount + + ", keyPageCount=" + keyPageCount + + ", blobPageCount=" + blobPageCount + + ", metadataLookupMissCount=" + metadataLookupMissCount + + ", metadataLookupHitCount=" + metadataLookupHitCount + + ", metadataSize=" + metadataSize + + ", findKeyTimer=" + findKeyTimer + + ", flushedKeyCount=" + flushedKeyCount + + ", flushCount=" + flushCount + + ", lookups=" + lookups + + ", maxLookupSize=" + maxLookupSize + + '}'; + } + + public String present(PartitionStats previous) { + PartitionStats deltaStats = this.minus(previous); + + long lookupCount = Math.max(1, deltaStats.metadataLookupHitCount + deltaStats.metadataLookupMissCount); + return "PartitionStats{ Deltas: " + + "MDPages=" + deltaStats.metadataPageCount + + ", KeyPages=" + deltaStats.keyPageCount + + ", BlobPages=" + deltaStats.blobPageCount + + ", NewKeys=" + deltaStats.metadataLookupMissCount + + ", ExistingKeys=" + deltaStats.metadataLookupHitCount + + ", MeanLookupTime=" + deltaStats.findKeyTimer / (lookupCount * 1000)+ "us" + + ", flushedKeys=" + deltaStats.flushedKeyCount + + ", flushCount=" + deltaStats.flushCount + + "; Totals:" + + "MeanLookupSize=" + metadataSize / Math.max(lookups, 1) + + ", MaxLookupSize=" + maxLookupSize + + "}"; + } + + public PartitionStats minus(PartitionStats other) { + if (Objects.isNull(other)) throw new NullPointerException("PartitionStats minus method argument is null"); + return new PartitionStats( + metadataPageCount - other.metadataPageCount, + keyPageCount - other.keyPageCount, + blobPageCount - other.blobPageCount, + metadataLookupMissCount - other.metadataLookupMissCount, + metadataLookupHitCount - other.metadataLookupHitCount, + metadataSize - other.metadataSize, + findKeyTimer - other.findKeyTimer, + flushedKeyCount - other.flushedKeyCount, + flushCount - other.flushCount, + lookups - other.lookups, + maxLookupSize - other.maxLookupSize + ); + } + + public PartitionStats add(PartitionStats other) { + if (Objects.isNull(other)) throw new NullPointerException("PartitionStats add method argument is null"); + return new PartitionStats( + metadataPageCount + other.metadataPageCount, + keyPageCount + other.keyPageCount, + blobPageCount + other.blobPageCount, + metadataLookupMissCount + other.metadataLookupMissCount, + metadataLookupHitCount + other.metadataLookupHitCount, + metadataSize + other.metadataSize, + findKeyTimer + other.findKeyTimer, + flushedKeyCount + other.flushedKeyCount, + flushCount + other.flushCount, + lookups + other.lookups, + Math.max(maxLookupSize, other.maxLookupSize) + ); + } +} diff --git a/src/main/java/com/upserve/uppend/ReadOnlyAppendOnlyStore.java b/src/main/java/com/upserve/uppend/ReadOnlyAppendOnlyStore.java index ef4e1051..660e346c 100644 --- a/src/main/java/com/upserve/uppend/ReadOnlyAppendOnlyStore.java +++ b/src/main/java/com/upserve/uppend/ReadOnlyAppendOnlyStore.java @@ -1,7 +1,5 @@ package com.upserve.uppend; -import com.github.benmanes.caffeine.cache.stats.CacheStats; - import java.util.Map; import java.util.function.BiConsumer; import java.util.stream.Stream; @@ -68,15 +66,9 @@ public interface ReadOnlyAppendOnlyStore extends Trimmable, AutoCloseable { */ void scan(BiConsumer> callback); - CacheStats getBlobPageCacheStats(); - - CacheStats getKeyPageCacheStats(); - - CacheStats getLookupKeyCacheStats(); - - CacheStats getMetadataCacheStats(); - BlockStats getBlockLongStats(); + PartitionStats getPartitionStats(); + long keyCount(); } diff --git a/src/main/java/com/upserve/uppend/ReadOnlyCounterStore.java b/src/main/java/com/upserve/uppend/ReadOnlyCounterStore.java index 13cbfcf5..56ee645a 100644 --- a/src/main/java/com/upserve/uppend/ReadOnlyCounterStore.java +++ b/src/main/java/com/upserve/uppend/ReadOnlyCounterStore.java @@ -1,7 +1,5 @@ package com.upserve.uppend; -import com.github.benmanes.caffeine.cache.stats.CacheStats; - import java.util.Map; import java.util.function.ObjLongConsumer; import java.util.stream.Stream; @@ -41,11 +39,5 @@ public interface ReadOnlyCounterStore extends Trimmable, AutoCloseable { */ void scan(ObjLongConsumer callback); - CacheStats getKeyPageCacheStats(); - - CacheStats getLookupKeyCacheStats(); - - CacheStats getMetadataCacheStats(); - long keyCount(); } diff --git a/src/main/java/com/upserve/uppend/blobs/BulkAppender.java b/src/main/java/com/upserve/uppend/blobs/BulkAppender.java deleted file mode 100644 index 94d1b33f..00000000 --- a/src/main/java/com/upserve/uppend/blobs/BulkAppender.java +++ /dev/null @@ -1,39 +0,0 @@ -package com.upserve.uppend.blobs; - -import java.util.concurrent.atomic.AtomicLong; - -public class BulkAppender { - - private final AtomicLong filePosition; - private final long initialPosition; - private final VirtualPageFileIO virtualPageFileIO; - - private final byte[] bulkBytes; - - - public BulkAppender(VirtualPageFileIO virtualPageFileIO, int bulkWriteSize) { - filePosition = new AtomicLong(virtualPageFileIO.getPosition()); - initialPosition = filePosition.get(); - - this.virtualPageFileIO = virtualPageFileIO; - - bulkBytes = new byte[bulkWriteSize]; - } - - public long getBulkAppendPosition(long size) { - return filePosition.getAndAdd(size); - } - - public void addBulkAppendBytes(long pos, byte[] bytes) { - System.arraycopy(bytes, 0, bulkBytes, (int) (pos - initialPosition), bytes.length); - } - - public void finishBulkAppend() { - final long writePosition = virtualPageFileIO.appendPosition(bulkBytes.length); - if (writePosition + bulkBytes.length != filePosition.get()) { - throw new IllegalStateException("Bulk appender position and length do not match"); - } - - virtualPageFileIO.write(writePosition, bulkBytes); - } -} diff --git a/src/main/java/com/upserve/uppend/blobs/FilePage.java b/src/main/java/com/upserve/uppend/blobs/FilePage.java index 428c5c7a..887227f3 100644 --- a/src/main/java/com/upserve/uppend/blobs/FilePage.java +++ b/src/main/java/com/upserve/uppend/blobs/FilePage.java @@ -7,7 +7,7 @@ import static java.lang.Integer.min; /** - * Mapped Byte Buffer backed implementation of Page + * File backed implementation of Page */ public class FilePage implements Page { @@ -30,10 +30,7 @@ public class FilePage implements Page { @Override public int get(int pagePosition, byte[] dst, int bufferOffset) { - final int desiredRead = dst.length - bufferOffset; - final int availableToRead = pageSize - pagePosition; - - final int actualRead = min(desiredRead, availableToRead); + final int actualRead = actualOperationSize(pagePosition, pageSize, bufferOffset, dst.length); // Make a local buffer with local position ByteBuffer byteBuffer = ByteBuffer.wrap(dst, bufferOffset, actualRead); @@ -52,9 +49,7 @@ public int get(int pagePosition, byte[] dst, int bufferOffset) { @Override public int put(int pagePosition, byte[] src, int bufferOffset) { - final int desiredWrite = src.length - bufferOffset; - final int availableToWrite = pageSize - pagePosition; - final int actualWrite = min(desiredWrite, availableToWrite); + final int actualWrite = actualOperationSize(pagePosition, pageSize, bufferOffset, src.length); // Make a local buffer with local position ByteBuffer byteBuffer = ByteBuffer.wrap(src, bufferOffset, actualWrite); @@ -70,5 +65,4 @@ public int put(int pagePosition, byte[] src, int bufferOffset) { return actualWrite; } - -} +} \ No newline at end of file diff --git a/src/main/java/com/upserve/uppend/blobs/MappedPage.java b/src/main/java/com/upserve/uppend/blobs/MappedPage.java index 1354c9fb..0466a2a5 100644 --- a/src/main/java/com/upserve/uppend/blobs/MappedPage.java +++ b/src/main/java/com/upserve/uppend/blobs/MappedPage.java @@ -2,36 +2,43 @@ import java.nio.*; -import static java.lang.Integer.min; - /** * Mapped Byte Buffer backed implementation of Page */ public class MappedPage implements Page { - private final MappedByteBuffer buffer; private final int pageSize; + private final int startingPosition; /** * Constructor for a MappedPage * * @param buffer the mapped byte buffer representing a page of a file + * @param startingPosition the starting offset in a larger buffer + * @param pageSize the size of the page to create */ - public MappedPage(MappedByteBuffer buffer) { - this.pageSize = buffer.capacity(); + public MappedPage(MappedByteBuffer buffer, int startingPosition, int pageSize) { + this.pageSize = pageSize; this.buffer = buffer; + this.startingPosition = startingPosition; + } + + /** + * Constructor for a MappedPage + * + * @param buffer the mapped byte buffer representing a page of a file + */ + public MappedPage(MappedByteBuffer buffer) { + this(buffer, 0, buffer.capacity()); } @Override public int get(int pagePosition, byte[] dst, int bufferOffset) { - final int desiredRead = dst.length - bufferOffset; - final int availableToRead = pageSize - pagePosition; - - final int actualRead = min(desiredRead, availableToRead); + final int actualRead = actualOperationSize(pagePosition, pageSize, bufferOffset, dst.length); // Make a local buffer with local position ByteBuffer localBuffer = buffer.duplicate(); - localBuffer.position(pagePosition); + localBuffer.position(pagePosition + startingPosition); localBuffer.get(dst, bufferOffset, actualRead); return actualRead; @@ -39,16 +46,13 @@ public int get(int pagePosition, byte[] dst, int bufferOffset) { @Override public int put(int pagePosition, byte[] src, int bufferOffset) { - final int desiredWrite = src.length - bufferOffset; - final int availableToWrite = pageSize - pagePosition; - final int actualWrite = min(desiredWrite, availableToWrite); + final int actualWrite = actualOperationSize(pagePosition, pageSize, bufferOffset, src.length); // Make a local buffer with local position ByteBuffer localBuffer = buffer.duplicate(); - localBuffer.position(pagePosition); + localBuffer.position(pagePosition + startingPosition); localBuffer.put(src, bufferOffset, actualWrite); return actualWrite; } - } diff --git a/src/main/java/com/upserve/uppend/blobs/Page.java b/src/main/java/com/upserve/uppend/blobs/Page.java index af47b5f7..3885e596 100644 --- a/src/main/java/com/upserve/uppend/blobs/Page.java +++ b/src/main/java/com/upserve/uppend/blobs/Page.java @@ -1,5 +1,7 @@ package com.upserve.uppend.blobs; +import static java.lang.Integer.min; + /** * Currently only MappedByteBuffer and FilePage are implemented - could implement on heap caching using * read once byte[]! @@ -8,4 +10,10 @@ public interface Page { int get(int pagePosition, byte[] dst, int bufferOffset); int put(int pagePosition, byte[] src, int bufferOffset); + + default int actualOperationSize(int pagePosition, int pagesize, int bufferOffset, int bufferLength) { + final int desiredRead = bufferLength - bufferOffset; + final int availableToRead = pagesize - pagePosition; + return min(desiredRead, availableToRead); + } } diff --git a/src/main/java/com/upserve/uppend/blobs/PageCache.java b/src/main/java/com/upserve/uppend/blobs/PageCache.java deleted file mode 100644 index 902bee8e..00000000 --- a/src/main/java/com/upserve/uppend/blobs/PageCache.java +++ /dev/null @@ -1,68 +0,0 @@ -package com.upserve.uppend.blobs; - -import com.github.benmanes.caffeine.cache.*; -import com.github.benmanes.caffeine.cache.stats.*; -import org.slf4j.Logger; - -import java.io.Flushable; -import java.lang.invoke.MethodHandles; -import java.nio.file.Path; -import java.util.Optional; -import java.util.concurrent.ExecutorService; -import java.util.function.*; - -/** - * A cache of memory mapped file pages - *

- * Concurrent writes to random new pages cause the JVM to crash in JDK less than 9 - * Attempted solution to force the buffer when it extends the file failed to fix the issue. See testHammerPageCache - * Uppend should not make concurrent writes to multiple pages in normal operation - unless blobs are larger than a page - */ -public class PageCache implements Flushable { - private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - private final Cache pageCache; - private final int pageSize; - - public PageCache(int pageSize, int initialCacheSize, int maximumCacheSize, ExecutorService executorService, Supplier metricsSupplier) { - this.pageSize = pageSize; - - Caffeine cacheBuilder = Caffeine - .newBuilder() - .executor(executorService) - .initialCapacity(initialCacheSize) - .maximumSize(maximumCacheSize) - .removalListener((key, value, cause) -> { - log.debug("Called removal on {} with cause {}", key, cause); - }); - - if (metricsSupplier != null) { - cacheBuilder = cacheBuilder.recordStats(metricsSupplier); - } - - this.pageCache = cacheBuilder - .build(); - - } - - public int getPageSize() { - return pageSize; - } - - Page get(long pos, Path path, Function pageLoader) { - return pageCache.get(new PageKey(path, pos), pageLoader); - } - - Optional getIfPresent(VirtualPageFile virtualPageFile, long pos) { - return Optional.ofNullable(pageCache.getIfPresent(new PageKey(virtualPageFile.getFilePath(), pos))); - } - - public CacheStats stats() { - return pageCache.stats(); - } - - @Override - public void flush() { - pageCache.invalidateAll(); - } -} diff --git a/src/main/java/com/upserve/uppend/blobs/PageKey.java b/src/main/java/com/upserve/uppend/blobs/PageKey.java deleted file mode 100644 index 247624b7..00000000 --- a/src/main/java/com/upserve/uppend/blobs/PageKey.java +++ /dev/null @@ -1,44 +0,0 @@ -package com.upserve.uppend.blobs; - -import java.nio.file.Path; -import java.util.Objects; - -public class PageKey { - private final Path filePath; - private final long position; - - PageKey(Path filePath, long position) { - this.filePath = filePath; - this.position = position; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - PageKey pageKey = (PageKey) o; - return position == pageKey.position && - Objects.equals(filePath, pageKey.filePath); - } - - @Override - public int hashCode() { - return Objects.hash(filePath, position); - } - - Path getFilePath() { - return filePath; - } - - long getPosition() { - return position; - } - - @Override - public String toString() { - return "PageKey{" + - "filePath=" + filePath + - ", position=" + position + - '}'; - } -} diff --git a/src/main/java/com/upserve/uppend/blobs/VirtualAppendOnlyBlobStore.java b/src/main/java/com/upserve/uppend/blobs/VirtualAppendOnlyBlobStore.java index 10bd83db..c29b9d22 100644 --- a/src/main/java/com/upserve/uppend/blobs/VirtualAppendOnlyBlobStore.java +++ b/src/main/java/com/upserve/uppend/blobs/VirtualAppendOnlyBlobStore.java @@ -22,6 +22,14 @@ public long getPosition() { return super.getPosition(); } + /** + * Read a byte array at this position from the virtual blob store + * Results are unpredictable for bad position requests. It may lead to a negative size and a NegativeArraySizeException + * it may lead to an IllegalStateException if the page does not yet exist for that position or it may result in an + * empty array value if the page exists but the position is currently past the end of the virtual file. + * @param pos the position to read from in the virtual file + * @return the byte array blob + */ public byte[] read(long pos) { if (log.isTraceEnabled()) log.trace("read mapped from {} @ {}", virtualFileNumber, pos); int size = readInt(pos); @@ -32,11 +40,11 @@ public byte[] read(long pos) { return buf; } - public static int recordSize(byte[] inputBytes) { + private static int recordSize(byte[] inputBytes) { return inputBytes.length + 4; } - public static byte[] byteRecord(byte[] inputBytes) { + private static byte[] byteRecord(byte[] inputBytes) { byte[] result = new byte[recordSize(inputBytes)]; System.arraycopy(int2bytes(inputBytes.length), 0, result, 0, 4); System.arraycopy(inputBytes, 0, result, 4, inputBytes.length); diff --git a/src/main/java/com/upserve/uppend/blobs/VirtualLongBlobStore.java b/src/main/java/com/upserve/uppend/blobs/VirtualLongBlobStore.java index 58cdff4a..600142cf 100644 --- a/src/main/java/com/upserve/uppend/blobs/VirtualLongBlobStore.java +++ b/src/main/java/com/upserve/uppend/blobs/VirtualLongBlobStore.java @@ -1,6 +1,5 @@ package com.upserve.uppend.blobs; - import com.google.common.collect.Maps; import org.slf4j.Logger; @@ -16,7 +15,7 @@ public class VirtualLongBlobStore extends VirtualPageFileIO { private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); public VirtualLongBlobStore(int virtualFileNumber, VirtualPageFile virtualPageFile) { - super(virtualFileNumber, virtualPageFile, true); + super(virtualFileNumber, virtualPageFile); } public long append(long val, byte[] bytes) { @@ -77,11 +76,11 @@ public Map.Entry next() { }; } - public static int recordSize(byte[] inputBytes) { + private static int recordSize(byte[] inputBytes) { return inputBytes.length + 12; } - public static byte[] byteRecord(long val, byte[] inputBytes) { + private static byte[] byteRecord(long val, byte[] inputBytes) { byte[] result = new byte[recordSize(inputBytes)]; System.arraycopy(int2bytes(inputBytes.length), 0, result, 0, 4); System.arraycopy(long2bytes(val), 0, result, 4, 8); diff --git a/src/main/java/com/upserve/uppend/blobs/VirtualMutableBlobStore.java b/src/main/java/com/upserve/uppend/blobs/VirtualMutableBlobStore.java index e38ff6c1..0143db77 100644 --- a/src/main/java/com/upserve/uppend/blobs/VirtualMutableBlobStore.java +++ b/src/main/java/com/upserve/uppend/blobs/VirtualMutableBlobStore.java @@ -43,11 +43,11 @@ public byte[] read(long pos) { } } - public static int recordSize(byte[] inputBytes) { + private static int recordSize(byte[] inputBytes) { return inputBytes.length + 8; } - public static byte[] byteRecord(byte[] inputBytes) { + private static byte[] byteRecord(byte[] inputBytes) { byte[] result = new byte[recordSize(inputBytes)]; System.arraycopy(int2bytes(inputBytes.length), 0, result, 0, 4); System.arraycopy(hashFunction.hashBytes(inputBytes).asBytes(), 0, result, 4, 4); diff --git a/src/main/java/com/upserve/uppend/blobs/VirtualPageFile.java b/src/main/java/com/upserve/uppend/blobs/VirtualPageFile.java index 92e83b77..0649cc19 100644 --- a/src/main/java/com/upserve/uppend/blobs/VirtualPageFile.java +++ b/src/main/java/com/upserve/uppend/blobs/VirtualPageFile.java @@ -1,6 +1,6 @@ package com.upserve.uppend.blobs; -import com.upserve.uppend.util.ThreadLocalByteBuffers; +import com.upserve.uppend.util.*; import org.slf4j.Logger; import java.io.*; @@ -8,7 +8,7 @@ import java.nio.*; import java.nio.channels.FileChannel; import java.nio.file.*; -import java.util.Arrays; +import java.util.*; import java.util.concurrent.atomic.*; import java.util.function.Supplier; import java.util.stream.IntStream; @@ -20,13 +20,15 @@ *

* Self Describing Header: virtualFiles (int), pageSize (int) *

+ * Page Table Locations (tables are interspersed with pages after the first block) + *

* Header: - * (long, long, long int) - * VF1 firstPageStart, lastPageStart, currentPosition, pageCount - * VF2 firstPageStart, lastPageStart, currentPosition, pageCount - * VF3 firstPageStart, lastPageStart, currentPosition, pageCount + * (long int) + * VF1 currentPosition, pageCount + * VF2 currentPosition, pageCount + * VF3 currentPosition, pageCount *

- * PageStart Table (long): + * PageTable (long): * VF1, VF2, VF3, VF4,... VIRTUAL_FILES * Page1 ....... .................. * Page2 ....... .................. @@ -36,40 +38,45 @@ * ... ....... .................. * PAGES_PER_VIRUAL_FILE *

- * Pages: - * previousPageStart(long), pageSize(bytes), nextPageStart(long) + * Pages - a collection of bytes of size pageSize *

- * A fixed number of pages per virtual file are allocated at startup - exceeding this number would be... bad - * TODO - fix this! + * Pages are interspersed with additional Page Tables as needed */ -public class VirtualPageFile implements Flushable, Closeable { +public class VirtualPageFile implements Closeable { private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private static final Supplier LOCAL_LONG_BUFFER = ThreadLocalByteBuffers.LOCAL_LONG_BUFFER; private static final Supplier LOCAL_INT_BUFFER = ThreadLocalByteBuffers.LOCAL_INT_BUFFER; + private static final int SELF_DESCRIBING_HEADER_SIZE = 8; - private static final int HEADER_RECORD_SIZE = 8 + 8 + 8 + 4; - /* firstPageStart, lastPageStart, currentPosition, pageCount */ + private static final int MAX_PAGE_TABLE_BLOCKS = 1024; // The number of Page Tables + private static final int PAGE_TABLE_BLOCK_LOCATION_HEADER_SIZE = MAX_PAGE_TABLE_BLOCKS * 8; // Storage for the locations of the page tables + + private static final int HEADER_RECORD_SIZE = 8 + 4; + /* currentPosition, pageCount */ + + // Maximum number of pages per record block + private static final int PAGE_TABLE_SIZE = 1000; - // Maximum number of pages allowed per virtual file - private static final int PAGES_PER_VIRUAL_FILE = 1000; + private static final int MAX_BUFFERS = 1024 * 64; // 128 TB per partition for 2Gb Bufffers + private final MappedByteBuffer[] mappedByteBuffers; + private final int bufferSize; - private final Path filePath; + final Path filePath; private final FileChannel channel; + private final LongBuffer headerBlockLocations; private final MappedByteBuffer headerBuffer; - private final MappedByteBuffer pageTableBuffer; private final AtomicLong nextPagePosition; private final boolean readOnly; - private final AtomicLong[] lastPagePositions; // the position in the physical file of the last page for each virtual file - private final AtomicLong[] firstPagePositions; // the position in the physical file of first page for each virtual file private final AtomicLong[] virtualFilePositions; // the current position in the virtual file for each virtual file private final AtomicInteger[] virtualFilePageCounts; // the number of pages currently allocated for each virtual file - private final LongBuffer pageTable; // Indexable list of page start locations for each virtual file + private final LongAdder pageAllocationCount; + + private final LongBuffer[] pageTables; // Array of Index-able list of page start locations for each virtual file private final int virtualFiles; private final int pageSize; @@ -77,18 +84,9 @@ public class VirtualPageFile implements Flushable, Closeable { private final int headerSize; private final int tableSize; - private final PageCache pageCache; - private final FileChannel.MapMode mapMode; - - // Public methods - public VirtualPageFile(Path filePath, int virtualFiles, int pageSize, boolean readOnly) { - this(filePath, virtualFiles, pageSize, readOnly, null); - } - - public VirtualPageFile(Path filePath, int virtualFiles, boolean readOnly, PageCache pageCache) { - this(filePath, virtualFiles, pageCache.getPageSize(), readOnly, pageCache); - } + private final int totalHeaderSize; + private final FileChannel.MapMode mapMode; public Path getFilePath() { return filePath; @@ -96,18 +94,16 @@ public Path getFilePath() { @Override public void close() throws IOException { - flush(); - channel.close(); - } + if (!channel.isOpen()) return; + Arrays.fill(mappedByteBuffers, null); - @Override - public void flush() throws IOException { - headerBuffer.force(); - pageTableBuffer.force(); - channel.force(true); + if (!readOnly) { + channel.truncate(nextPagePosition.get()); + } + channel.close(); } - public int getVirtualFiles() { + int getVirtualFiles() { return virtualFiles; } @@ -115,6 +111,10 @@ public boolean isReadOnly() { return readOnly; } + public int getAllocatedPageCount() { + return pageAllocationCount.intValue(); + } + // Package private methods boolean isPageAvailable(int virtualFileNumber, int pageNumber) { if (readOnly) { @@ -198,21 +198,19 @@ int pagePosition(long pos) { */ int pageNumber(long pos) { long result = (pos / (long) pageSize); - if (result >= PAGES_PER_VIRUAL_FILE) - throw new IllegalStateException("The position " + pos + " exceeds the page limit for file" + getFilePath()); + if (result >= PAGE_TABLE_SIZE * MAX_PAGE_TABLE_BLOCKS) + throw new IllegalStateException("The position " + pos + " exceeds the page limit " + PAGE_TABLE_SIZE * MAX_PAGE_TABLE_BLOCKS + ", for file" + getFilePath() + "with page size " + pageSize ); return (int) result; } /** * Get or create (allocate) the page if it does not exist. - * Returns a MappedByteBuffer backed Page if there is one in the cache. Otherwise it returns a FilePage * * @param virtualFileNumber the virtual file number * @param pageNumber the page number to getValue - * @param useMapped indicates whether to use a memory mapped file page or a file channel backed page * @return a Page for File IO */ - Page getCachedOrCreatePage(int virtualFileNumber, int pageNumber, boolean useMapped) { + Page getOrCreatePage(int virtualFileNumber, int pageNumber) { final long startPosition; if (isPageAvailable(virtualFileNumber, pageNumber)) { startPosition = getValidPageStart(virtualFileNumber, pageNumber); @@ -220,64 +218,65 @@ Page getCachedOrCreatePage(int virtualFileNumber, int pageNumber, boolean useMap startPosition = allocatePosition(virtualFileNumber, pageNumber); } - if (pageCache != null) { - return pageCache.get(startPosition, getFilePath(), pageKey -> page(pageKey.getPosition(), useMapped)); - } else { - return page(startPosition, useMapped); - } + return filePage(startPosition); } /** - * Get a MappedByteBuffer backed Page uses a page cache if present - always uses memory mapped pages + * Get the existing page * * @param virtualFileNumber the virtual file number * @param pageNumber the page number to getValue * @return a Page for File IO */ Page getExistingPage(int virtualFileNumber, int pageNumber) { - // TODO we could cache each page start in the writer if reading the long from the mapped byte buffer gets expensive long startPosition = getValidPageStart(virtualFileNumber, pageNumber); + return mappedPage(startPosition); + } - if (pageCache != null) { - return pageCache.get(startPosition, getFilePath(), pageKey -> mappedPage(pageKey.getPosition())); - } else { - return mappedPage(startPosition); - } + private MappedPage mappedPage(long startPosition) { + final long postHeaderPosition = startPosition - (totalHeaderSize); + final int mapIndex = (int) (postHeaderPosition / bufferSize); + final int mapPosition = (int) (postHeaderPosition % bufferSize); + + MappedByteBuffer bigbuffer = ensureBuffered(mapIndex); + + return new MappedPage(bigbuffer, mapPosition, pageSize); } - Page page(long startPosition, boolean useMapped) { - if (useMapped) { - return mappedPage(startPosition); - } else { - return filePage(startPosition); - } + private FilePage filePage(long startPosition) { + return new FilePage(channel, startPosition, pageSize); } - MappedPage mappedPage(long startPosition) { + long getFileSize(){ try { - return new MappedPage(channel.map(mapMode, startPosition + 8, pageSize)); + return channel.size(); } catch (IOException e) { - throw new UncheckedIOException("Unable to map page from file " + filePath, e); + throw new UncheckedIOException("Could not get file size for:" + filePath, e); } } - FilePage filePage(long startPosition) { - return new FilePage(channel, startPosition + 8, pageSize); - } - - // Private methods - private VirtualPageFile(Path filePath, int virtualFiles, int pageSize, boolean readOnly, PageCache pageCache) { + public VirtualPageFile(Path filePath, int virtualFiles, int pageSize, int targetBufferSize, boolean readOnly) { this.filePath = filePath; this.readOnly = readOnly; this.virtualFiles = virtualFiles; this.pageSize = pageSize; - this.pageCache = pageCache; + + this.mappedByteBuffers = new MappedByteBuffer[MAX_BUFFERS]; + + if (targetBufferSize < (pageSize)) throw new IllegalArgumentException("Target buffer size " + targetBufferSize + " must be larger than a page " + pageSize); + + this.bufferSize = (targetBufferSize / (pageSize)) * (pageSize); + + log.debug("Using buffer size " + bufferSize + " with page size " + pageSize); if (virtualFiles < 1) throw new IllegalArgumentException("virtualFiles must be greater than 0 in file: " + filePath); + headerSize = virtualFiles * HEADER_RECORD_SIZE; + tableSize = virtualFiles * PAGE_TABLE_SIZE * 8; + OpenOption[] openOptions; if (readOnly) { - openOptions = new OpenOption[]{StandardOpenOption.READ}; + openOptions = new OpenOption[]{StandardOpenOption.READ, StandardOpenOption.WRITE}; mapMode = FileChannel.MapMode.READ_ONLY; } else { openOptions = new OpenOption[]{StandardOpenOption.CREATE, StandardOpenOption.READ, StandardOpenOption.WRITE}; @@ -290,17 +289,20 @@ private VirtualPageFile(Path filePath, int virtualFiles, int pageSize, boolean r throw new UncheckedIOException("Unable to open file: " + filePath, e); } - + final long initialSize; try { - long initialSize = channel.size(); - ByteBuffer intBuffer = LOCAL_INT_BUFFER.get(); + initialSize = channel.size(); + headerBlockLocations = channel.map(mapMode, SELF_DESCRIBING_HEADER_SIZE, PAGE_TABLE_BLOCK_LOCATION_HEADER_SIZE).asLongBuffer(); + ByteBuffer intBuffer = LOCAL_INT_BUFFER.get(); if (!readOnly && initialSize == 0) { intBuffer.putInt(virtualFiles); channel.write(intBuffer.flip(), 0); intBuffer.flip().putInt(pageSize); channel.write(intBuffer.flip(), 4); + + headerBlockLocations.put(0, SELF_DESCRIBING_HEADER_SIZE + PAGE_TABLE_BLOCK_LOCATION_HEADER_SIZE + headerSize); } else { channel.read(intBuffer, 0); int val = intBuffer.flip().getInt(); @@ -311,33 +313,23 @@ private VirtualPageFile(Path filePath, int virtualFiles, int pageSize, boolean r val = intBuffer.flip().getInt(); if (val != virtualFiles) throw new IllegalArgumentException("The specfied page size " + pageSize + " does not match the value in the datastore " + val + " in file " + getFilePath()); + + long longVal = headerBlockLocations.get(0); + if (longVal != SELF_DESCRIBING_HEADER_SIZE + PAGE_TABLE_BLOCK_LOCATION_HEADER_SIZE + headerSize) + throw new IllegalArgumentException("The header sizes " + (SELF_DESCRIBING_HEADER_SIZE + PAGE_TABLE_BLOCK_LOCATION_HEADER_SIZE + headerSize) + " does not match the value in the datastore " + longVal + " in file " + getFilePath()); } } catch (IOException e) { - throw new UncheckedIOException("Unable to get file size" + " in file " + getFilePath(), e); + throw new UncheckedIOException("Unable to read, write, map or get the size of " + getFilePath(), e); } - - headerSize = virtualFiles * HEADER_RECORD_SIZE; - tableSize = virtualFiles * PAGES_PER_VIRUAL_FILE * 8; + totalHeaderSize = headerSize + tableSize + SELF_DESCRIBING_HEADER_SIZE + PAGE_TABLE_BLOCK_LOCATION_HEADER_SIZE; try { - headerBuffer = channel.map(mapMode, SELF_DESCRIBING_HEADER_SIZE, headerSize); + headerBuffer = channel.map(mapMode, SELF_DESCRIBING_HEADER_SIZE + PAGE_TABLE_BLOCK_LOCATION_HEADER_SIZE, headerSize); } catch (IOException e) { throw new UncheckedIOException("unable to map header for path: " + filePath, e); } - firstPagePositions = IntStream - .range(0, virtualFiles) - .mapToLong(this::getHeaderFirstPage) - .mapToObj(AtomicLong::new) - .toArray(AtomicLong[]::new); - - lastPagePositions = IntStream - .range(0, virtualFiles) - .mapToLong(this::getHeaderLastPage) - .mapToObj(AtomicLong::new) - .toArray(AtomicLong[]::new); - virtualFilePositions = IntStream .range(0, virtualFiles) .mapToLong(this::getHeaderVirtualFilePosition) @@ -350,112 +342,82 @@ private VirtualPageFile(Path filePath, int virtualFiles, int pageSize, boolean r .mapToObj(AtomicInteger::new) .toArray(AtomicInteger[]::new); - long lastStartPosition = Arrays.stream(lastPagePositions).mapToLong(AtomicLong::get).max().orElse(0L); + // For statistics... + pageAllocationCount = new LongAdder(); + pageAllocationCount.add(Arrays.stream(virtualFilePageCounts).mapToLong(AtomicInteger::get).sum()); + pageTables = new LongBuffer[MAX_PAGE_TABLE_BLOCKS]; try { - pageTableBuffer = channel.map(mapMode, headerSize + SELF_DESCRIBING_HEADER_SIZE, tableSize); - pageTable = pageTableBuffer.asLongBuffer(); + pageTables[0] = channel + .map(mapMode, headerSize + SELF_DESCRIBING_HEADER_SIZE + PAGE_TABLE_BLOCK_LOCATION_HEADER_SIZE, tableSize) + .asLongBuffer(); } catch (IOException e) { throw new UncheckedIOException("unable to map page locations for path: " + filePath, e); } - if (lastStartPosition == 0) { - nextPagePosition = new AtomicLong(headerSize + tableSize + SELF_DESCRIBING_HEADER_SIZE); - - } else if (lastStartPosition < headerSize + tableSize + SELF_DESCRIBING_HEADER_SIZE) { - throw new IllegalStateException("file position " + lastStartPosition + " is less than header size: " + headerSize + " in file " + filePath); - } else { - nextPagePosition = new AtomicLong(lastStartPosition + pageSize + 16); - } - - IntStream.range(0, virtualFiles).parallel().forEach(this::detectCorruption); - - // TODO Can we fix corruption instead of just bailing? - } - - private void detectCorruption(int virtualFileNumber) { - - long virtualFilePosition = virtualFilePositions[virtualFileNumber].get(); - int pageCount = virtualFilePageCounts[virtualFileNumber].get(); - long firstPageStart = firstPagePositions[virtualFileNumber].get(); - long finalPageStart = lastPagePositions[virtualFileNumber].get(); - - long[] pageStarts = IntStream.range(0, PAGES_PER_VIRUAL_FILE).mapToLong(page -> getRawPageStart(virtualFileNumber, page)).toArray(); - - if (pageCount == 0) { - if (virtualFilePosition != 0 || firstPageStart != 0 || finalPageStart != 0 || Arrays.stream(pageStarts).anyMatch(val -> val != 0)) { - throw new IllegalStateException("None zero positions for file with no pages in file " + getFilePath()); + long lastTableStart = 0; + for(int i=0; i< MAX_PAGE_TABLE_BLOCKS; i++) { + long position = headerBlockLocations.get(i); + if (position > 0){ + lastTableStart = position; } - return; } - if (virtualFilePosition / pageSize > pageCount) - throw new IllegalStateException("The current virtual file position is outside the last page in file " + getFilePath()); - - if (firstPageStart != pageStarts[0]) - throw new IllegalStateException("Header first pageStart does not match table page 0 start in file " + getFilePath()); - if (finalPageStart != pageStarts[pageCount - 1]) - throw new IllegalStateException("Header last pageStart does not match table page last start in file " + getFilePath()); - - long nextPageStart = firstPageStart; - long lastPageStart = -1L; - for (int page = 0; page < pageCount; page++) { - if (nextPageStart != pageStarts[page]) - throw new IllegalStateException("Head pointer does not match table page start in file " + getFilePath()); - if (readTailPointer(nextPageStart) != lastPageStart) - throw new IllegalStateException("Corrupt tail pointer in first page in file " + getFilePath()); + if (lastTableStart + tableSize > Math.max(initialSize, totalHeaderSize)) { + throw new IllegalStateException("Bad value for last table start in header"); + } - lastPageStart = nextPageStart; + long lastStartPosition = IntStream + .range(0, virtualFiles) + .mapToLong(index -> { + int pageCount = virtualFilePageCounts[index].get(); + int pageIndex = pageCount > 0 ? pageCount - 1 : 0; + return getRawPageStart(index, pageIndex); + }) + .max().orElse(0L); - nextPageStart = readHeadPointer(nextPageStart); + if (lastStartPosition == 0) { + nextPagePosition = new AtomicLong(totalHeaderSize); + } else if (lastStartPosition < totalHeaderSize) { + throw new IllegalStateException("file position " + lastStartPosition + " is less than header size: " + headerSize + " in file " + filePath); + } else { + nextPagePosition = new AtomicLong(Math.max(lastStartPosition + pageSize, lastTableStart + tableSize)); + preloadBuffers(nextPagePosition.get()); } - - if (nextPageStart != -1) throw new IllegalStateException("Last head pointer not equal -1 in file " + getFilePath()); } private long getRawPageStart(int virtualFileNumber, int pageNumber) { - return pageTable.get(PAGES_PER_VIRUAL_FILE * virtualFileNumber + pageNumber); + int pageTableNumber = pageNumber / PAGE_TABLE_SIZE; + int pageInTable = pageNumber % PAGE_TABLE_SIZE; + + return ensurePageTable(pageTableNumber).get(PAGE_TABLE_SIZE * virtualFileNumber + pageInTable); } private long getValidPageStart(int virtualFileNumber, int pageNumber) { - if (pageNumber == -1) return -1L; long result = getRawPageStart(virtualFileNumber, pageNumber); - if (result < headerSize + tableSize + SELF_DESCRIBING_HEADER_SIZE) { - throw new IllegalStateException("Invalid page position " + result + " is in the file header; in page table for file " + virtualFileNumber + " page " + pageNumber + " in file " + getFilePath()); - } - if ((result - (headerSize + tableSize + SELF_DESCRIBING_HEADER_SIZE)) % (pageSize + 16) != 0 ) { - throw new IllegalStateException("Invalid page position " + result + " is not aligned with pageSize " + pageSize + " + 16 ; in page table for file " + virtualFileNumber + " page " + pageNumber + " in file " + getFilePath()); + if (result < totalHeaderSize) { + if (result == 0) { + throw new IllegalStateException("The page start position is zero for page " + pageNumber + " in file " + virtualFileNumber + ": this typically means it has not yet been allocated"); + } else { + throw new IllegalStateException("Invalid page start position " + result + " is in the file header; bad value in page table for virtual file " + virtualFileNumber + " in page " + pageNumber + " in file " + getFilePath()); + } } return result; } private void putPageStart(int virtualFileNumber, int pageNumber, long position) { - int index = PAGES_PER_VIRUAL_FILE * virtualFileNumber + pageNumber; - pageTable.put(index, position); - } - - private long getHeaderFirstPage(int virtualFileNumber) { - return headerBuffer.getLong(virtualFileNumber * HEADER_RECORD_SIZE); - } - - private void putHeaderFirstPage(int virtualFileNumber, long position) { - headerBuffer.putLong(virtualFileNumber * HEADER_RECORD_SIZE, position); - } + int pageTableNumber = pageNumber / PAGE_TABLE_SIZE; + int pageInTable = pageNumber % PAGE_TABLE_SIZE; - private long getHeaderLastPage(int virtualFileNumber) { - return headerBuffer.getLong(virtualFileNumber * HEADER_RECORD_SIZE + 8); - } - - private void putHeaderLastPage(int virtualFileNumber, long position) { - headerBuffer.putLong(virtualFileNumber * HEADER_RECORD_SIZE + 8, position); + ensurePageTable(pageTableNumber).put(PAGE_TABLE_SIZE * virtualFileNumber + pageInTable, position); } private long getHeaderVirtualFilePosition(int virtualFileNumber) { - return headerBuffer.getLong(virtualFileNumber * HEADER_RECORD_SIZE + 16); + return headerBuffer.getLong(virtualFileNumber * HEADER_RECORD_SIZE); } private void putHeaderVirtualFilePosition(int virtualFileNumber, long position) { - headerBuffer.putLong(virtualFileNumber * HEADER_RECORD_SIZE + 16, position); + headerBuffer.putLong(virtualFileNumber * HEADER_RECORD_SIZE, position); } private AtomicLong getAtomicVirtualFilePosition(int virtualFileNumber) { @@ -463,88 +425,108 @@ private AtomicLong getAtomicVirtualFilePosition(int virtualFileNumber) { } private int getHeaderVirtualFilePageCount(int virtualFileNumber) { - return headerBuffer.getInt(virtualFileNumber * HEADER_RECORD_SIZE + 24); + return headerBuffer.getInt(virtualFileNumber * HEADER_RECORD_SIZE + 8); } private void putHeaderVirtualFilePageCount(int virtualFileNumber, int count) { - headerBuffer.putInt(virtualFileNumber * HEADER_RECORD_SIZE + 24, count); + headerBuffer.putInt(virtualFileNumber * HEADER_RECORD_SIZE + 8, count); } private long allocatePosition(int virtualFileNumber, int pageNumber) { synchronized (virtualFilePageCounts[virtualFileNumber]) { // If another thread has already done it - just return the start position - while (pageNumber >= virtualFilePageCounts[virtualFileNumber].get()) { - allocatePage(virtualFileNumber); + final int currentPageCount = virtualFilePageCounts[virtualFileNumber].get(); + if (pageNumber >= currentPageCount) { + allocatePage(virtualFileNumber, currentPageCount, pageNumber); } } return getValidPageStart(virtualFileNumber, pageNumber); } - private void allocatePage(int virtualFileNumber) { + private void allocatePage(int virtualFileNumber, int currentPageCount, int pageNumber) { + int pagesToAllocate = pageNumber - currentPageCount + 1; // Do the atomic stuff - long newPageStart = nextPagePosition.getAndAdd(pageSize + 16); - int newPageNumber = virtualFilePageCounts[virtualFileNumber].get(); // the result is the index, the value incremented at the end of the method is the new count! - boolean firstPage = firstPagePositions[virtualFileNumber].compareAndSet(0L, newPageStart); - lastPagePositions[virtualFileNumber].set(newPageStart); - - long lastPageStart = getValidPageStart(virtualFileNumber, newPageNumber - 1); - - if (lastPageStart > 0) writeHeadPointer(lastPageStart, newPageStart); - writeTailPointer(newPageStart, lastPageStart); - writeHeadPointer(newPageStart, -1); // Extends the file to the end of the page - - // Update the persistent table of pages - putPageStart(virtualFileNumber, newPageNumber, newPageStart); + long firstPageStart = nextPagePosition.getAndAdd(pageSize * pagesToAllocate); - // Update the persisted header values - if (firstPage) putHeaderFirstPage(virtualFileNumber, newPageStart); - putHeaderLastPage(virtualFileNumber, newPageStart); - putHeaderVirtualFilePageCount(virtualFileNumber, newPageNumber + 1); - - // Now that the page is allocated and persistent - update the counter which is the lock controlling access - virtualFilePageCounts[virtualFileNumber].getAndIncrement(); - } - - private void writeTailPointer(long pageStart, long previousPageStart) { - ByteBuffer longBuffer = LOCAL_LONG_BUFFER.get(); - longBuffer.asLongBuffer().put(previousPageStart); // will be -1 if this is the first page - try { - channel.write(longBuffer, pageStart); - } catch (IOException e) { - throw new UncheckedIOException("Unable to write tail pointer at " + pageStart + " in " + filePath, e); + for (int i=0; i < pagesToAllocate; i++) { + // Update the persistent table of pages + putPageStart(virtualFileNumber, currentPageCount + i, firstPageStart + i * pageSize); + // Now that the page is allocated and persistent - update the counter which is the lock controlling access } - } - private long readTailPointer(long pageStart) { - ByteBuffer longBuffer = LOCAL_LONG_BUFFER.get(); - - try { - channel.read(longBuffer, pageStart); - } catch (IOException e) { - throw new UncheckedIOException("Unable to read tail pointer at " + pageStart + " in " + filePath, e); + putHeaderVirtualFilePageCount(virtualFileNumber, currentPageCount + pagesToAllocate); + virtualFilePageCounts[virtualFileNumber].set(currentPageCount + pagesToAllocate); + + // Stats only + pageAllocationCount.add(pagesToAllocate); + } + + private MappedByteBuffer ensureBuffered(int bufferIndex) { + MappedByteBuffer buffer = mappedByteBuffers[bufferIndex]; + if (buffer == null) { + synchronized (mappedByteBuffers) { + buffer = mappedByteBuffers[bufferIndex]; + if (buffer == null) { + long bufferStart = ((long) bufferIndex * bufferSize) + totalHeaderSize; + try { + buffer = channel.map(FileChannel.MapMode.READ_ONLY, bufferStart, bufferSize); + } catch (IOException e) { + throw new UncheckedIOException("Unable to map buffer for index " + bufferIndex + " at (" + bufferStart + " start position) in file " + filePath, e); + } + mappedByteBuffers[bufferIndex] = buffer; + } + } } - return longBuffer.flip().getLong(); - } - - private void writeHeadPointer(long pageStart, long nextPageStart) { - ByteBuffer longBuffer = LOCAL_LONG_BUFFER.get(); - longBuffer.asLongBuffer().put(nextPageStart); // will be -1 if this is the first page - try { - channel.write(longBuffer, pageStart + pageSize + 8); - } catch (IOException e) { - throw new UncheckedIOException("Unable to write head pointer at " + pageStart + " in " + filePath, e); + return buffer; + } + + private LongBuffer ensurePageTable(int pageNumber) { + LongBuffer buffer = pageTables[pageNumber]; + if (buffer == null) { + synchronized (pageTables) { + buffer = pageTables[pageNumber]; + if (buffer == null) { + + long bufferStart = headerBlockLocations.get(pageNumber); + + // All allocated space must be in multiples of pageSize to guarantee a buffer will not end in the middle of a page + final int apparentSize; + if (pageSize > tableSize) { + apparentSize = pageSize; + } else { + apparentSize = (tableSize / pageSize + 1) * pageSize; + } + + if (!readOnly && bufferStart == 0) { + bufferStart = nextPagePosition.getAndAdd(apparentSize); + headerBlockLocations.put(pageNumber, bufferStart); + } + try { + buffer = channel.map(mapMode, bufferStart, tableSize).asLongBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Unable to map buffer for page table " + pageNumber + " at (" + bufferStart + " start position) in file " + filePath, e); + } + pageTables[pageNumber] = buffer; + } + } } + return buffer; } - private long readHeadPointer(long pageStart) { - ByteBuffer longBuffer = LOCAL_LONG_BUFFER.get(); + // Called during initialize only - no need to synchronize + private void preloadBuffers(long nextPagePosition){ + for (int bufferIndex=0; bufferIndex= nextPagePosition) break; + + try { + MappedByteBuffer buffer = channel.map(FileChannel.MapMode.READ_ONLY, bufferStart, bufferSize); + mappedByteBuffers[bufferIndex] = buffer; + } catch (IOException e) { + throw new UncheckedIOException("Unable to preload mapped buffer for index " + bufferIndex + " at (" + bufferStart + " start position) in file " + filePath, e); + } } - return longBuffer.flip().getLong(); } } diff --git a/src/main/java/com/upserve/uppend/blobs/VirtualPageFileIO.java b/src/main/java/com/upserve/uppend/blobs/VirtualPageFileIO.java index b227243a..9f7e961f 100644 --- a/src/main/java/com/upserve/uppend/blobs/VirtualPageFileIO.java +++ b/src/main/java/com/upserve/uppend/blobs/VirtualPageFileIO.java @@ -6,30 +6,20 @@ import java.lang.invoke.MethodHandles; import java.nio.ByteBuffer; -import java.util.concurrent.atomic.AtomicStampedReference; import java.util.function.Supplier; public class VirtualPageFileIO { private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - static final Supplier LOCAL_INT_BUFFER = ThreadLocalByteBuffers.LOCAL_INT_BUFFER; - static final Supplier LOCAL_LONG_BUFFER = ThreadLocalByteBuffers.LOCAL_LONG_BUFFER; + private static final Supplier LOCAL_INT_BUFFER = ThreadLocalByteBuffers.LOCAL_INT_BUFFER; + private static final Supplier LOCAL_LONG_BUFFER = ThreadLocalByteBuffers.LOCAL_LONG_BUFFER; - protected final int virtualFileNumber; + final int virtualFileNumber; private final VirtualPageFile virtualPageFile; - private final boolean useMappedPages; - private final AtomicStampedReference lastPage; VirtualPageFileIO(int virtualFileNumber, VirtualPageFile virtualPageFile) { - this(virtualFileNumber, virtualPageFile, false); - } - - VirtualPageFileIO(int virtualFileNumber, VirtualPageFile virtualPageFile, boolean useMappedPages) { this.virtualFileNumber = virtualFileNumber; this.virtualPageFile = virtualPageFile; - this.useMappedPages = useMappedPages; - - lastPage = new AtomicStampedReference<>(null, -1); if (virtualFileNumber > virtualPageFile.getVirtualFiles()) throw new IllegalStateException("Requested a virtual file " + virtualFileNumber + " which is greater than the max allocated " + virtualPageFile.getVirtualFiles()); @@ -82,7 +72,7 @@ static byte[] long2bytes(long val) { void write(long pos, byte[] bytes) { if (bytes.length == 0) { - throw new IllegalArgumentException("Can not write empty bytes!"); + throw new IllegalStateException("Can not write empty bytes to pos " + pos + " in file " + virtualFileNumber + " of " + virtualPageFile.filePath); } final int result = writePagedOffset(pos, bytes, 0); if (result != bytes.length) { @@ -93,13 +83,7 @@ void write(long pos, byte[] bytes) { private int writePagedOffset(long pos, byte[] bytes, int offset) { int pageNumber = virtualPageFile.pageNumber(pos); - Page page; - int[] holder = new int[1]; - page = lastPage.get(holder); - if (holder[0] != pageNumber) { - page = virtualPageFile.getCachedOrCreatePage(virtualFileNumber, pageNumber, useMappedPages); - lastPage.set(page, pageNumber); - } + Page page = virtualPageFile.getOrCreatePage(virtualFileNumber, pageNumber); int bytesWritten; bytesWritten = page.put(virtualPageFile.pagePosition(pos), bytes, offset); @@ -125,6 +109,7 @@ long readLong(long pos) { } void read(long pos, byte[] buf) { + // Short circuit read of empty value here if (buf.length == 0) return; final int result = readPagedOffset(pos, buf, 0); if (result != buf.length) { @@ -135,19 +120,12 @@ void read(long pos, byte[] buf) { private int readPagedOffset(long pos, byte[] buf, int offset) { int pageNumber = virtualPageFile.pageNumber(pos); - Page page; - int[] holder = new int[1]; - page = lastPage.get(holder); - if (holder[0] != pageNumber) { - page = virtualPageFile.getExistingPage(virtualFileNumber, pageNumber); - lastPage.set(page, pageNumber); - } + Page page = virtualPageFile.getExistingPage(virtualFileNumber, pageNumber); int bytesRead; bytesRead = page.get(virtualPageFile.pagePosition(pos), buf, offset); if (bytesRead < (buf.length - offset)) { - // TODO see if it is faster to use the head pointer to the next page start rather than recursion here? bytesRead += readPagedOffset(pos + bytesRead, buf, offset + bytesRead); } return bytesRead; diff --git a/src/main/java/com/upserve/uppend/cli/Cli.java b/src/main/java/com/upserve/uppend/cli/Cli.java index c98e3d59..7e1f1d9f 100644 --- a/src/main/java/com/upserve/uppend/cli/Cli.java +++ b/src/main/java/com/upserve/uppend/cli/Cli.java @@ -1,15 +1,28 @@ package com.upserve.uppend.cli; +import org.slf4j.Logger; import picocli.CommandLine; import picocli.CommandLine.*; +import java.lang.invoke.MethodHandles; import java.nio.file.*; import java.util.concurrent.Callable; @SuppressWarnings("WeakerAccess") @Command( name = "uppend", - description = "An append-only, key-multivalue store", + header = { + "@|green ooooo ooo .o8 |@", + "@|green `888' `8' 888 |@", + "@|green 888 8 oo.ooooo. oo.ooooo. .ooooo. ooo. .oo. .oooo888 |@", + "@|green 888 8 888' `88b 888' `88b d88' `88b `888P'Y88b d88' `888 |@", + "@|green 888 8 888 888 888 888 888ooo888 888 888 888 888 |@", + "@|green `88. .8' 888 888 888 888 888 .o 888 888 888 888 |@", + "@|green `YbodP' 888bod8P' 888bod8P' `Y8bod8P' o888o o888o `Y8bod88P |@", + "@|green 888 888 |@", + "@|green o888o o888o |@", + }, + description = "Uppend is an append-only, key-multivalue store", synopsisHeading = "%nUsage: ", parameterListHeading = "%nParameters:%n", optionListHeading = "%nOptions:%n", @@ -17,11 +30,14 @@ descriptionHeading = "%n", footerHeading = "%n", subcommands = { + CommandVersion.class, CommandBenchmark.class, - CommandVersion.class + CommandFileStoreBenchmark.class } ) public class Cli implements Callable { + private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + @SuppressWarnings("unused") @Option(names = "--help", usageHelp = true, description = "Print usage") boolean help; @@ -33,9 +49,8 @@ public Void call() throws Exception { } public static void main(String... args) throws Exception { - CommandLine cmd = new CommandLine(new Cli()); - cmd.registerConverter(Path.class, (p) -> Paths.get(p)); - // TODO how to redirect to errStream? - cmd.parseWithHandler(new RunLast(), args); + new CommandLine(new Cli()) + .registerConverter(Path.class, (p) -> Paths.get(p)) + .execute(args); } } diff --git a/src/main/java/com/upserve/uppend/cli/CommandBenchmark.java b/src/main/java/com/upserve/uppend/cli/CommandBenchmark.java index e24c5aa4..16b8d1a1 100644 --- a/src/main/java/com/upserve/uppend/cli/CommandBenchmark.java +++ b/src/main/java/com/upserve/uppend/cli/CommandBenchmark.java @@ -27,6 +27,7 @@ public class CommandBenchmark implements Callable { public static final String ROOT_NAME = "Root"; public static final String STORE_NAME = "Benchmark"; + Benchmark benchmark; @Parameters(index = "0", description = "Store path") Path path; @@ -40,8 +41,8 @@ public class CommandBenchmark implements Callable { @Option(names = {"-c", "--case"}, description = "Benchmark class (narrow|wide) key space") BenchmarkCase benchmarkCase = BenchmarkCase.narrow; - @Option(names= {"-i", "--iostat"}, description = "arguments for iostat process") - String ioStatArgs = "5"; + @Option(names = {"-b", "--buffer-size"}, description = "Buffer Size (small|medium|large)") + BufferSize bufferSize = BufferSize.medium; @SuppressWarnings("unused") @Option(names = "--help", usageHelp = true, description = "Print usage") @@ -54,22 +55,18 @@ public Void call() throws Exception { log.warn("Location already exists: appending to {}", path); } - Benchmark benchmark = createBenchmark(); + benchmark = createBenchmark(); benchmark.run(); return null; } - private Benchmark createBenchmark() { long keys; long count; final int blockSize; - int partitions; - int hashSize; - - int keyCacheSize; - long keyCacheWeight; + int partitionCount; + int hashCount; int blobCacheSize; int blobPageSize; @@ -90,21 +87,11 @@ private Benchmark createBenchmark() { keys = (long) Math.pow(Math.log10(count), 2.0) * 100; blockSize = 16_384; - partitions = 64; - hashSize = 64; + partitionCount = 64; + hashCount = 64; - // Cache all the keys - keyCacheSize = (int) keys; - keyCacheWeight = keys * 9 + 1000; // 9 bytes per key plus some room - - blobCacheSize = 16_384; blobPageSize = 16 * 1024 * 1024; - - keyPageCacheSize = 16 * partitions * hashSize; keyPageSize = 1024 * 1024; - - metadataCacheSize = partitions * hashSize; - metadataCacheWeight = keys * 4 + 1000; // one int per key plus some room metadataPageSize = 1024 * 1024; flushDelay = 60; @@ -117,23 +104,15 @@ private Benchmark createBenchmark() { count = keys * 2; blockSize = 4; - hashSize = 512; - partitions = 128; - keyCacheSize = 0; - keyCacheWeight = 0; - - blobCacheSize = 524_288; - blobPageSize = 1024 * 1024; // Pages will roll over at 135M keys + hashCount = 256; + partitionCount = 128; - keyPageCacheSize = 16 * partitions * hashSize; - keyPageSize = 1024 * 1024; // Key pages will roll over at about 2.9B keys - - metadataCacheSize = partitions * hashSize; - metadataCacheWeight = 4 * keys + 10_000; // one int per key plus some room - metadataPageSize = 1024 * 1024; + blobPageSize = 64 * 1024; // Pages will roll over at 135M keys + keyPageSize = 4 * 1024; // Key pages will roll over at about 2.9B keys + metadataPageSize = 4 * 1024; flushDelay = -1; - flushThreshold = 512; + flushThreshold = 256; break; @@ -146,32 +125,17 @@ private Benchmark createBenchmark() { AppendOnlyStoreBuilder builder = Uppend.store(path) .withStoreName(STORE_NAME) .withMetricsRootName(ROOT_NAME) - .withBlobsPerBlock(blockSize) - .withLongLookupHashSize(hashSize) - .withPartitionSize(partitions) // Use direct partition - - .withInitialLookupKeyCacheSize(keyCacheSize) - .withMaximumLookupKeyCacheWeight(keyCacheWeight) - - .withInitialBlobCacheSize(blobCacheSize) - .withMaximumBlobCacheSize(blobCacheSize) + .withLongLookupHashCount(hashCount) + .withPartitionCount(partitionCount) // Use direct partition + .withTargetBufferSize(bufferSize.getSize()) .withBlobPageSize(blobPageSize) - - .withInitialLookupPageCacheSize(keyPageCacheSize) - .withMaximumLookupPageCacheSize(keyPageCacheSize) .withLookupPageSize(keyPageSize) - - .withInitialMetaDataCacheSize(metadataCacheSize) - .withMetaDataPageSize(metadataPageSize) - .withMaximumMetaDataCacheWeight(metadataCacheWeight) - + .withMetadataPageSize(metadataPageSize) .withFlushThreshold(flushThreshold) .withFlushDelaySeconds(flushDelay) + .withStoreMetrics(metrics); - .withStoreMetrics(metrics) - .withCacheMetrics(); - - return new Benchmark(mode, builder, partitions, keys, count, ioStatArgs); + return new Benchmark(mode, builder, keys, count); } } diff --git a/src/main/java/com/upserve/uppend/cli/CommandFileStoreBenchmark.java b/src/main/java/com/upserve/uppend/cli/CommandFileStoreBenchmark.java new file mode 100644 index 00000000..6cb14e8d --- /dev/null +++ b/src/main/java/com/upserve/uppend/cli/CommandFileStoreBenchmark.java @@ -0,0 +1,87 @@ +package com.upserve.uppend.cli; + +import com.upserve.uppend.blobs.*; +import com.upserve.uppend.cli.benchmark.*; +import org.slf4j.Logger; +import picocli.CommandLine; + +import java.lang.invoke.MethodHandles; +import java.nio.file.*; +import java.util.*; +import java.util.concurrent.*; +import java.util.stream.IntStream; + +@SuppressWarnings({"WeakerAccess", "unused"}) +@CommandLine.Command( + name = "filestore", + description = "Run file store benchmark", + showDefaultValues = true, + synopsisHeading = "%nUsage: uppend ", + parameterListHeading = "%nParameters:%n", + optionListHeading = "%nOptions:%n", + commandListHeading = "%nCommands:%n", + descriptionHeading = "%n", + footerHeading = "%n" +) +public class CommandFileStoreBenchmark implements Callable { + private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + private IntSummaryStatistics result; + + @CommandLine.Parameters(index = "0", description = "Store path") + Path path; + + @CommandLine.Option(names= {"-n", "--virtual-files"}, description = "the number of virtual files to use") + int nfiles = 32; + + @CommandLine.Option(names = {"-b", "--buffer-size"}, description = "Buffer Size (small|medium|large)") + BufferSize bufferSize = BufferSize.medium; + + @CommandLine.Option(names = {"-s", "--size"}, description = "Benchmark size (nano|micro|small|medium|large|huge|gigantic)") + BenchmarkSize size = BenchmarkSize.medium; + + @CommandLine.Option(names = {"-p", "--page-size"}, description = "Page Size (small|medium|large)") + PageSize pageSize = PageSize.medium; + + @SuppressWarnings("unused") + @CommandLine.Option(names = "--help", usageHelp = true, description = "Print usage") + boolean help; + + public IntSummaryStatistics getStats() { + return result; + } + + @Override + public Void call() throws Exception { + + log.info("FileStore Benchmark"); + + Random random = new Random(); + + Files.createDirectories(path.getParent()); + + VirtualPageFile file = new VirtualPageFile(path, nfiles, pageSize.getSize(), bufferSize.getSize(), false); + VirtualAppendOnlyBlobStore[] stores = IntStream.range(0, nfiles) + .mapToObj(val -> new VirtualAppendOnlyBlobStore(val, file)) + .toArray(VirtualAppendOnlyBlobStore[]::new); + + + long tic = System.currentTimeMillis(); + + result = ThreadLocalRandom.current() + .ints(size.getSize(), 0, nfiles) + .parallel() + .map(val -> { + + byte[] bytes = new byte[ThreadLocalRandom.current().nextInt(1024)]; + random.nextBytes(bytes); + + stores[val].append(bytes); + return bytes.length; + }).summaryStatistics(); + + long toc = System.currentTimeMillis(); + + log.info("[All Done!] {} ms", toc - tic); + return null; + } +} diff --git a/src/main/java/com/upserve/uppend/cli/benchmark/Benchmark.java b/src/main/java/com/upserve/uppend/cli/benchmark/Benchmark.java index 9d08435d..c60f1e7f 100644 --- a/src/main/java/com/upserve/uppend/cli/benchmark/Benchmark.java +++ b/src/main/java/com/upserve/uppend/cli/benchmark/Benchmark.java @@ -2,19 +2,15 @@ import com.codahale.metrics.*; import com.codahale.metrics.Timer; -import com.github.benmanes.caffeine.cache.stats.CacheStats; import com.upserve.uppend.*; -import com.upserve.uppend.lookup.FlushStats; import org.slf4j.Logger; import java.io.IOException; import java.lang.invoke.MethodHandles; -import java.nio.file.*; import java.util.*; import java.util.concurrent.*; import java.util.concurrent.atomic.*; import java.util.function.Supplier; -import java.util.stream.LongStream; import static com.upserve.uppend.AutoFlusher.forkJoinPoolFunction; import static com.upserve.uppend.cli.CommandBenchmark.ROOT_NAME; @@ -24,55 +20,50 @@ public class Benchmark { private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private Runnable writer; - private Runnable reader; - - private final Random random = new Random(); + private BenchmarkRunnable writer; + private BenchmarkRunnable reader; private BenchmarkMode mode; private final MetricRegistry metrics; private long range; private long count; - private int maxPartitions; - private long maxKeys; private int sleep = 0; + private int partitionCount; + private int hashCount; private final AppendOnlyStore testInstance; private final ForkJoinPool writerPool; private final ForkJoinPool readerPool; - private final ForkJoinPool cachePool; + AtomicReference partitionStats; + AtomicReference blockStats; + + public LongSummaryStatistics writerStats() { + return writer.getStats(); + } - private volatile boolean isDone = false; - private final String ioStatArgs; + public LongSummaryStatistics readerStats() { + return reader.getStats(); + } - public Benchmark(BenchmarkMode mode, AppendOnlyStoreBuilder builder, int maxPartitions, long maxKeys, long count, String ioStatArgs) { + public Benchmark(BenchmarkMode mode, AppendOnlyStoreBuilder builder, long range, long count) { this.mode = mode; this.count = count; - this.maxPartitions = maxPartitions; // max ~ 2000 - this.maxKeys = maxKeys; // max ~ 100,000,000 + this.range = range; - this.ioStatArgs = ioStatArgs; + partitionCount = builder.getPartitionCount(); + hashCount = builder.getLookupHashCount(); writerPool = forkJoinPoolFunction.apply("benchmark-writer"); readerPool = forkJoinPoolFunction.apply("benchmark-reader"); - cachePool = forkJoinPoolFunction.apply("cache"); - - builder.withLookupPageCacheExecutorService(cachePool) - .withLookupMetaDataCacheExecutorService(cachePool) - .withBlobCacheExecutorService(cachePool) - .withLookupKeyCacheExecutorService(cachePool); - metrics = builder.getStoreMetricsRegistry(); log.info(builder.toString()); - range = (long) maxKeys; - switch (mode) { case readwrite: testInstance = builder.build(false); @@ -96,16 +87,19 @@ public Benchmark(BenchmarkMode mode, AppendOnlyStoreBuilder builder, int maxPart case scan: testInstance = builder.build(true); writer = BenchmarkWriter.noop(); - reader = scanReader(testInstance); + reader = new ScanReader(testInstance); break; default: throw new RuntimeException("Unknown mode: " + mode); } + + partitionStats = new AtomicReference<>(testInstance.getPartitionStats()); + blockStats = new AtomicReference<>(testInstance.getBlockLongStats()); } private BenchmarkWriter simpleWriter() { return new BenchmarkWriter( - random.longs(count, 0, range).parallel(), + ThreadLocalRandom.current().longs(count, 0, range).parallel(), longInt -> { byte[] myBytes = bytes(longInt); String formatted = format(longInt); @@ -117,7 +111,7 @@ private BenchmarkWriter simpleWriter() { private BenchmarkReader simpleReader() { return new BenchmarkReader( - random.longs(count, 0, range).parallel(), + ThreadLocalRandom.current().longs(count, 0, range).parallel(), longInt -> { String formatted = format(longInt); return testInstance.read(formatted, formatted) @@ -127,11 +121,25 @@ private BenchmarkReader simpleReader() { ); } - private Runnable scanReader(AppendOnlyStore appendOnlyStore) { - return () -> { - long count = appendOnlyStore.scan().mapToLong(entry -> entry.getValue().count()).sum(); + private class ScanReader implements BenchmarkRunnable{ + private final AppendOnlyStore appendOnlyStore; + private LongSummaryStatistics result; + + public ScanReader(AppendOnlyStore appendOnlyStore) { + this.appendOnlyStore = appendOnlyStore; + } + + @Override + public LongSummaryStatistics getStats() { + return result; + } + + + @Override + public void run() { + result = appendOnlyStore.scan().mapToLong(entry -> entry.getValue().count()).summaryStatistics(); log.info("Scanned {} entries", count); - }; + } } public static String format(long value) { @@ -161,7 +169,6 @@ private TimerTask watcherTimer() { readBytesMeter = metrics.meter(MetricRegistry.name(ROOT_NAME, UPPEND_APPEND_STORE, STORE_NAME, READ_BYTES_METER_METRIC_NAME)); } - final Runtime runtime = Runtime.getRuntime(); AtomicLong tic = new AtomicLong(System.currentTimeMillis()); @@ -170,17 +177,10 @@ private TimerTask watcherTimer() { AtomicLong read = new AtomicLong(readBytesMeter.getCount()); AtomicLong readCount = new AtomicLong(readCounter.get()); - AtomicReference blobPageCacheStats = new AtomicReference(testInstance.getBlobPageCacheStats()); - AtomicReference keyPageCacheStats = new AtomicReference(testInstance.getKeyPageCacheStats()); - AtomicReference lookupKeyCacheStats = new AtomicReference(testInstance.getLookupKeyCacheStats()); - AtomicReference metadataCacheStats = new AtomicReference(testInstance.getMetadataCacheStats()); - AtomicReference flushStats = new AtomicReference(testInstance.getFlushStats()); - return new TimerTask() { @Override public void run() { long val; - CacheStats stats; try { val = System.currentTimeMillis(); double elapsed = (val - tic.getAndSet(val)) / 1000D; @@ -202,26 +202,11 @@ public void run() { log.info(String.format("Read: %7.2fmb/s %7.2fr/s; Write %7.2fmb/s %7.2fa/s; Mem %7.2fmb free %7.2fmb total", readRate, keysReadPerSecond, writeRate, appendsPerSecond, free, total)); - stats = testInstance.getBlobPageCacheStats(); - log.info("Blob Page Cache: {}", stats.minus(blobPageCacheStats.getAndSet(stats))); - - stats = testInstance.getKeyPageCacheStats(); - log.info("Key Page Cache: {}", stats.minus(keyPageCacheStats.getAndSet(stats))); - - stats = testInstance.getLookupKeyCacheStats(); - log.info("Lookup Key Cache: {}", stats.minus(lookupKeyCacheStats.getAndSet(stats))); - - stats = testInstance.getMetadataCacheStats(); - log.info("Metadata Cache: {}", stats.minus(metadataCacheStats.getAndSet(stats))); - - FlushStats fstats = testInstance.getFlushStats(); - log.info("Flush Stats: {}", fstats.minus(flushStats.getAndSet(fstats))); - - log.info("Cache Pool: {}", cachePool); - log.info("Write Pool: {}", writerPool); - log.info("Read Pool: {}", readerPool); - log.info("Common Pool: {}", ForkJoinPool.commonPool()); + PartitionStats pStats = testInstance.getPartitionStats(); + log.info(pStats.present(partitionStats.getAndSet(pStats))); + BlockStats bStats = testInstance.getBlockLongStats(); + log.info("Block Stats: {}", bStats.minus(blockStats.getAndSet(bStats))); } catch (Exception e) { @@ -232,13 +217,7 @@ public void run() { } public void run() throws InterruptedException, ExecutionException, IOException { - log.info("Running Performance test with {} partitions, {} keys and {} count", maxPartitions, maxKeys, count); - - ProcessBuilder processBuilder = new ProcessBuilder(("iostat " + ioStatArgs).split("\\s+")); - log.info("Running IOSTAT: '{}'", processBuilder.command()); - processBuilder.redirectErrorStream(true); - processBuilder.redirectOutput(ProcessBuilder.Redirect.INHERIT); - Process process = processBuilder.start(); + log.info("Running Performance test with {} partitions {} hashCount, {} keys and {} count", partitionCount, hashCount, range, count); Future writerFuture = writerPool.submit(writer); @@ -267,11 +246,6 @@ public void run() throws InterruptedException, ExecutionException, IOException { throw new RuntimeException("error closing test uppend store", e); } - process.destroy(); - log.info("Benchmark is All Done!"); - System.out.println("[benchmark is done]"); // used in CliTest - isDone = true; } } - diff --git a/src/main/java/com/upserve/uppend/cli/benchmark/BenchmarkReader.java b/src/main/java/com/upserve/uppend/cli/benchmark/BenchmarkReader.java index 6e38f915..035be87b 100644 --- a/src/main/java/com/upserve/uppend/cli/benchmark/BenchmarkReader.java +++ b/src/main/java/com/upserve/uppend/cli/benchmark/BenchmarkReader.java @@ -3,15 +3,22 @@ import org.slf4j.Logger; import java.lang.invoke.MethodHandles; +import java.util.LongSummaryStatistics; import java.util.function.Function; import java.util.stream.LongStream; -public class BenchmarkReader implements Runnable { +public class BenchmarkReader implements BenchmarkRunnable { private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private final LongStream longStream; private final Function longFunction; + private LongSummaryStatistics result; + + public LongSummaryStatistics getStats() { + return result; + } + BenchmarkReader(LongStream longStream, Function longFunction) { this.longStream = longStream; this.longFunction = longFunction; @@ -28,7 +35,7 @@ public void run() { } log.info("starting reader..."); long tic = -1 * System.currentTimeMillis(); - longStream.forEach(longFunction::apply); + result = longStream.map(longFunction::apply).summaryStatistics(); log.info( String.format( "done reading in %5.2f seconds", diff --git a/src/main/java/com/upserve/uppend/cli/benchmark/BenchmarkRunnable.java b/src/main/java/com/upserve/uppend/cli/benchmark/BenchmarkRunnable.java new file mode 100644 index 00000000..5a3dccbd --- /dev/null +++ b/src/main/java/com/upserve/uppend/cli/benchmark/BenchmarkRunnable.java @@ -0,0 +1,7 @@ +package com.upserve.uppend.cli.benchmark; + +import java.util.LongSummaryStatistics; + +public interface BenchmarkRunnable extends Runnable { + LongSummaryStatistics getStats(); +} diff --git a/src/main/java/com/upserve/uppend/cli/benchmark/BenchmarkWriter.java b/src/main/java/com/upserve/uppend/cli/benchmark/BenchmarkWriter.java index c28dd29c..10220e78 100644 --- a/src/main/java/com/upserve/uppend/cli/benchmark/BenchmarkWriter.java +++ b/src/main/java/com/upserve/uppend/cli/benchmark/BenchmarkWriter.java @@ -3,15 +3,22 @@ import org.slf4j.Logger; import java.lang.invoke.MethodHandles; +import java.util.LongSummaryStatistics; import java.util.function.Function; import java.util.stream.LongStream; -public class BenchmarkWriter implements Runnable { +public class BenchmarkWriter implements BenchmarkRunnable { private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private final LongStream longStream; private final Function longFunction; + public LongSummaryStatistics getStats() { + return result; + } + + private LongSummaryStatistics result; + BenchmarkWriter(LongStream longStream, Function longFunction) { this.longStream = longStream; this.longFunction = longFunction; @@ -28,7 +35,7 @@ public void run() { } log.info("starting writer..."); long tic = -1 * System.currentTimeMillis(); - longStream.forEach(longFunction::apply); + result = longStream.map(longFunction::apply).summaryStatistics(); log.info( String.format( "done writing in %5.2f seconds", diff --git a/src/main/java/com/upserve/uppend/cli/benchmark/BufferSize.java b/src/main/java/com/upserve/uppend/cli/benchmark/BufferSize.java new file mode 100644 index 00000000..b873e225 --- /dev/null +++ b/src/main/java/com/upserve/uppend/cli/benchmark/BufferSize.java @@ -0,0 +1,17 @@ +package com.upserve.uppend.cli.benchmark; + +public enum BufferSize { + small(16 * 1024 * 1024), + medium(128 * 1024 * 1024), + large(Integer.MAX_VALUE); + + private final int size; + + BufferSize(int val) { + this.size = val; + } + + public int getSize() { + return size; + } +} diff --git a/src/main/java/com/upserve/uppend/cli/benchmark/PageSize.java b/src/main/java/com/upserve/uppend/cli/benchmark/PageSize.java new file mode 100644 index 00000000..b1931cd6 --- /dev/null +++ b/src/main/java/com/upserve/uppend/cli/benchmark/PageSize.java @@ -0,0 +1,17 @@ +package com.upserve.uppend.cli.benchmark; + +public enum PageSize { + small(512 * 1024), + medium(4 * 1024 * 1024), + large(32 * 1024 * 1024); + + private final int size; + + PageSize(int val) { + this.size = val; + } + + public int getSize() { + return size; + } +} diff --git a/src/main/java/com/upserve/uppend/lookup/FlushStats.java b/src/main/java/com/upserve/uppend/lookup/FlushStats.java deleted file mode 100644 index 745fdc48..00000000 --- a/src/main/java/com/upserve/uppend/lookup/FlushStats.java +++ /dev/null @@ -1,41 +0,0 @@ -package com.upserve.uppend.lookup; - -import com.upserve.uppend.BlockStats; - -import java.util.Objects; - -public class FlushStats { - private final long flushedKeys; - private final long flushedLookups; - - public FlushStats(long flushedKeys, long flushedLookups){ - this.flushedKeys = flushedKeys; - this.flushedLookups = flushedLookups; - } - - public static FlushStats ZERO_STATS = new FlushStats(0,0); - - public FlushStats minus(FlushStats other) { - if (Objects.isNull(other)) throw new NullPointerException("FlushStats minus method argument is null"); - return new FlushStats( - flushedKeys - other.flushedKeys, - flushedLookups - other.flushedLookups - ); - } - - public FlushStats add(FlushStats other) { - if (Objects.isNull(other)) throw new NullPointerException("FlushStats add method argument is null"); - return new FlushStats( - flushedKeys - other.flushedKeys, - flushedLookups - other.flushedLookups - ); - } - - @Override - public String toString() { - return "FlushStats{" + - "flushedKeys=" + flushedKeys + - ", flushedLookups=" + flushedLookups + - '}'; - } -} diff --git a/src/main/java/com/upserve/uppend/lookup/LookupCache.java b/src/main/java/com/upserve/uppend/lookup/LookupCache.java deleted file mode 100644 index 3167ce61..00000000 --- a/src/main/java/com/upserve/uppend/lookup/LookupCache.java +++ /dev/null @@ -1,111 +0,0 @@ -package com.upserve.uppend.lookup; - -import com.github.benmanes.caffeine.cache.*; -import com.github.benmanes.caffeine.cache.stats.*; -import org.slf4j.Logger; - -import java.io.Flushable; -import java.lang.invoke.MethodHandles; -import java.util.concurrent.*; -import java.util.concurrent.atomic.LongAdder; -import java.util.function.*; - -public class LookupCache implements Flushable { - private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - // An LRU cache of Lookup Keys - private final Cache keyLongLookupCache; - - private final LoadingCache lookupMetaDataCache; - - private final boolean keyCacheActive; - - private final LongAdder keysFlushed; - private final LongAdder lookupsFlushed; - - public LookupCache(int initialKeyCapacity, long maximumKeyWeight, ExecutorService executorServiceKeyCache, Supplier keyCacheMetricsSupplier, int intialMetaDataCapacity, long maximumMetaDataWeight, int metadataTTL, ExecutorService executorServiceMetaDataCache, Supplier metadataCacheMetricsSupplier) { - - Caffeine keyCacheBuilder = Caffeine - .newBuilder() - .executor(executorServiceKeyCache) - .initialCapacity(initialKeyCapacity) - .maximumWeight(maximumKeyWeight) // bytes - .weigher((k, v) -> k.weight()); - - if (keyCacheMetricsSupplier != null) { - keyCacheBuilder = keyCacheBuilder.recordStats(keyCacheMetricsSupplier); - } - - keyCacheActive = maximumKeyWeight > 0; - - keyLongLookupCache = keyCacheBuilder.build(); - - keysFlushed = new LongAdder(); - lookupsFlushed = new LongAdder(); - - Caffeine metadataCacheBuilder = Caffeine - .newBuilder() - .executor(executorServiceMetaDataCache) - .initialCapacity(intialMetaDataCapacity) - .maximumWeight(maximumMetaDataWeight) - .weigher((k, v) -> v.weight()); - - if (metadataTTL > 0) { - metadataCacheBuilder.expireAfterWrite(metadataTTL, TimeUnit.SECONDS); - } - - if (metadataCacheMetricsSupplier != null) { - metadataCacheBuilder = metadataCacheBuilder.recordStats(metadataCacheMetricsSupplier); - } - - lookupMetaDataCache = metadataCacheBuilder - .build(LookupData::loadMetadata); - } - - public FlushStats getFlushStats() { - return new FlushStats(keysFlushed.longValue(), lookupsFlushed.longValue()); - } - - public void addFlushCount(long val){ - keysFlushed.add(val); - lookupsFlushed.increment(); - } - - public boolean isKeyCacheActive() { - return keyCacheActive; - } - - public void putLookup(PartitionLookupKey key, long val) { - keyLongLookupCache.put(key, val); - } - - public Long getLong(PartitionLookupKey lookupKey, Function cacheLoader) { - return keyLongLookupCache.get(lookupKey, cacheLoader); - } - - public LookupMetadata getMetadata(LookupData key) { - return lookupMetaDataCache.get(key, LookupData::loadMetadata); - } - - public void putMetadata(LookupData key, LookupMetadata value) { - lookupMetaDataCache.put(key, value); - } - - public CacheStats keyStats() { - if (keyCacheActive) { - return keyLongLookupCache.stats(); - } else { - return CacheStats.empty(); - } - } - - public CacheStats metadataStats() { - return lookupMetaDataCache.stats(); - } - - @Override - public void flush() { - lookupMetaDataCache.invalidateAll(); - keyLongLookupCache.invalidateAll(); - } -} diff --git a/src/main/java/com/upserve/uppend/lookup/LookupData.java b/src/main/java/com/upserve/uppend/lookup/LookupData.java index f9a3bb7f..ad05b45a 100644 --- a/src/main/java/com/upserve/uppend/lookup/LookupData.java +++ b/src/main/java/com/upserve/uppend/lookup/LookupData.java @@ -1,7 +1,7 @@ package com.upserve.uppend.lookup; import com.google.common.collect.Maps; -import com.upserve.uppend.AutoFlusher; +import com.upserve.uppend.*; import com.upserve.uppend.blobs.*; import org.slf4j.Logger; @@ -14,28 +14,31 @@ import java.util.function.*; import java.util.stream.*; -public class LookupData implements Flushable { +public class LookupData implements Flushable, Trimmable { private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private final PartitionLookupCache partitionLookupCache; - private final Function lookupFunction; - private final BiConsumer lookupBiConsumer; + private static final Random random = new Random(); private final AtomicInteger writeCacheCounter; + private final AtomicBoolean flushing; + private final AtomicBoolean firstFlush; private final int flushThreshold; + private final int firstFlushThreshold; + private final int reloadInterval; // The container for stuff we need to write - Only new keys can be in the write cache final ConcurrentHashMap writeCache; - // keys written but not yet written to the metadata live here + // keys written but not yet in the metadata live here final ConcurrentHashMap flushCache; + // Direct reference for writers + private AtomicReference atomicMetadataRef; - private final AtomicReference flushReference; - private final ReadWriteLock metadataLock; - private final Lock metadataReadLock; - private final Lock metadataWriteLock; - + // Timestamped references for readers + private final AtomicStampedReference timeStampedMetadata; + private final AtomicInteger reloadStamp; + private final long startTime; private final boolean readOnly; @@ -49,52 +52,57 @@ public class LookupData implements Flushable { // Flushing every 30 seconds, we can run for 2000 years before the metaDataGeneration hits INTEGER.MAX_VALUE private AtomicInteger metaDataGeneration; + private LongAdder findKeyTimer; + private int flushCounter; + private long flushedKeyCounter; + + public static LookupData lookupWriter(VirtualLongBlobStore keyLongBlobs, VirtualMutableBlobStore metadataBlobs, int flushThreshold){ + return new LookupData(keyLongBlobs, metadataBlobs, flushThreshold, -1, false); + } - public LookupData(VirtualLongBlobStore keyLongBlobs, VirtualMutableBlobStore metadataBlobs, PartitionLookupCache lookupCache, boolean readOnly) { - this(keyLongBlobs, metadataBlobs, lookupCache, -1, readOnly); + public static LookupData lookupReader(VirtualLongBlobStore keyLongBlobs, VirtualMutableBlobStore metadataBlobs, int reloadInterval){ + return new LookupData(keyLongBlobs, metadataBlobs, -1, reloadInterval, true); } - public LookupData(VirtualLongBlobStore keyLongBlobs, VirtualMutableBlobStore metadataBlobs, PartitionLookupCache lookupCache, int flushThreshold, boolean readOnly) { + private LookupData(VirtualLongBlobStore keyLongBlobs, VirtualMutableBlobStore metadataBlobs, int flushThreshold, int reloadInterval, boolean readOnly) { this.keyLongBlobs = keyLongBlobs; this.metadataBlobs = metadataBlobs; - this.partitionLookupCache = lookupCache; - if (lookupCache.isKeyCacheActive()) { - lookupFunction = key -> partitionLookupCache.getLong(key , this::findValueFor); - lookupBiConsumer = partitionLookupCache::putLookup; - } else { - lookupFunction = this::findValueFor; - lookupBiConsumer = (key, value) -> {}; - } - this.readOnly = readOnly; + this.firstFlush = new AtomicBoolean(true); + this.firstFlushThreshold = flushThreshold * (random.nextInt(100) + 25) / 100; + this.flushing = new AtomicBoolean(false); this.flushThreshold = flushThreshold; + this.reloadInterval = reloadInterval; + + // Record the time we started this LookupData + startTime = System.currentTimeMillis(); + + writeCacheCounter = new AtomicInteger(); + metaDataGeneration = new AtomicInteger(); + findKeyTimer = new LongAdder(); + + atomicMetadataRef = new AtomicReference<>(); if (readOnly) { writeCache = null; flushCache = null; - flushReference = null; - - metadataLock = null; - metadataReadLock = null; - metadataWriteLock = null; + // Reload interval is specified in seconds + timeStampedMetadata = new AtomicStampedReference<>(loadMetadata(), reloadInterval); + reloadStamp = new AtomicInteger(reloadInterval); } else { + atomicMetadataRef.set(loadMetadata()); + + timeStampedMetadata = null; + reloadStamp = null; + writeCache = new ConcurrentHashMap<>(); flushCache = new ConcurrentHashMap<>(); - flushReference = new AtomicReference<>(); - - metadataLock = new ReentrantReadWriteLock(); - metadataReadLock = metadataLock.readLock(); - metadataWriteLock = metadataLock.writeLock(); } - writeCacheCounter = new AtomicInteger(); - - metaDataGeneration = new AtomicInteger(); - consistentWriteCacheLock = new ReentrantReadWriteLock(); consistentWriteCacheReadLock = consistentWriteCacheLock.readLock(); consistentWriteCacheWriteLock = consistentWriteCacheLock.writeLock(); @@ -108,9 +116,15 @@ public LookupData(VirtualLongBlobStore keyLongBlobs, VirtualMutableBlobStore met */ public Long getValue(LookupKey key) { if (readOnly) { - return getCached(key); + return findValueFor(key); } else { - return writeCache.getOrDefault(key, getCached(key)); + Long result = writeCache.get(key); + + if (result == null) { + return findValueFor(key); + } else { + return result; + } } } @@ -119,11 +133,26 @@ VirtualMutableBlobStore getMetadataBlobs() { } private void flushThreshold() { - if (flushThreshold != -1 && writeCacheCounter.getAndIncrement() == flushThreshold) { + if (flushThreshold == -1) return; + + if (shouldFlush(writeCacheCounter.getAndIncrement())) { AutoFlusher.submitWork(this::flush); } } + private boolean shouldFlush(int writeCount) { + if (!flushing.get() && firstFlush.get() && writeCount == firstFlushThreshold) { + flushing.set(true); + firstFlush.set(false); + return true; + } else if (!flushing.get() && writeCount == flushThreshold){ + flushing.set(true); + return true; + } else { + return false; + } + } + /** * Set the value of the key if it does not exist and return the new value. If it does exist, return the existing value. * @@ -137,7 +166,7 @@ public long putIfNotExists(LookupKey key, LongSupplier allocateLongFunc) { long[] ref = new long[1]; writeCache.compute(key, (k, value) -> { if (value == null) { - Long existingValue = getCached(k); + Long existingValue = findValueFor(k); if (existingValue == null) { long val = allocateLongFunc.getAsLong(); ref[0] = val; @@ -170,7 +199,7 @@ public long putIfNotExists(LookupKey key, long value) { long[] ref = new long[1]; writeCache.compute(key, (k, val) -> { if (val == null) { - Long existingValue = getCached(k); + Long existingValue = findValueFor(k); if (existingValue == null) { ref[0] = value; flushThreshold(); @@ -202,7 +231,7 @@ public long increment(LookupKey key, long delta) { long[] ref = new long[1]; writeCache.compute(key, (writeKey, value) -> { if (value == null) { - Long existingValue = getCached(writeKey); + Long existingValue = findValueFor(writeKey); if (existingValue == null) { ref[0] = delta; flushThreshold(); @@ -213,7 +242,6 @@ public long increment(LookupKey key, long delta) { ref[0] = newValue; flushCache.computeIfPresent(writeKey, (flushKey, v) -> newValue); - lookupBiConsumer.accept(writeKey, newValue); // Update the read cache keyLongBlobs.writeLong(writeKey.getPosition(), newValue); // Update the value on disk // No need to add this to the write cache @@ -243,7 +271,7 @@ public Long put(LookupKey key, final long value) { Long[] ref = new Long[1]; writeCache.compute(key, (writeKey, val) -> { if (val == null) { - Long existingValue = getCached(writeKey); + Long existingValue = findValueFor(writeKey); if (existingValue == null) { ref[0] = null; flushThreshold(); @@ -253,7 +281,6 @@ public Long put(LookupKey key, final long value) { ref[0] = existingValue; flushCache.computeIfPresent(writeKey, (flushKey, v) -> value); - lookupBiConsumer.accept(writeKey, value); // Update the read cache keyLongBlobs.writeLong(writeKey.getPosition(), value); // Update the value on disk // No need to add this to the write cache @@ -269,14 +296,6 @@ public Long put(LookupKey key, final long value) { return ref[0]; } - private Long getCached(LookupKey key) { - if (readOnly) { - return lookupFunction.apply(key); - } else { - return flushCache.getOrDefault(key, lookupFunction.apply(key)); - } - } - /** * read the LookupKey by index * @@ -297,7 +316,6 @@ public Map.Entry readEntry(long keyPosition) { return Maps.immutableEntry(readKey(keyPosition), readValue(keyPosition)); } - /** * Read the long value associated with a particular key number * @@ -308,17 +326,26 @@ public long readValue(long keyPosition) { return keyLongBlobs.readLong(keyPosition); } - /** - * load a key from paged files for the partition lookup cache - * Must return null to prevent loading missing value into cache - * - * @param key the Key we are looking for - * @return Long value or null if not present - */ - private Long findValueFor(PartitionLookupKey key) { - return findValueFor(key.getLookupKey()); + public long getMetadataLookupMissCount(){ + return getMetadata().getMissCount(); + } + + public long getMetadataLookupHitCount(){ + return getMetadata().getHitCount(); + } + + public long getMetadataSize(){ + return getMetadata().getNumKeys(); + } + + public long getFindKeyTimer(){ + return findKeyTimer.sum(); } + public long getFlushedKeyCount() { return flushedKeyCounter; } + + public int getFlushCount() { return flushCounter; } + /** * Load a key from cached pages * @@ -326,7 +353,17 @@ private Long findValueFor(PartitionLookupKey key) { * @return Long value or null if not present */ private Long findValueFor(LookupKey key) { - return getMetadata().findKey(keyLongBlobs, key); + if (!readOnly) { + Long result = flushCache.get(key); + if (result != null ){ + return result; + } + } + LookupMetadata md = getMetadata(); + long tic = -System.nanoTime(); + Long val = md.findKey(keyLongBlobs, key); + findKeyTimer.add(System.nanoTime() + tic); + return val; } LookupMetadata loadMetadata() { @@ -334,21 +371,27 @@ LookupMetadata loadMetadata() { try { return LookupMetadata.open( getMetadataBlobs(), - getMetaDataGeneration() + getMetaDataGeneration(), + getMetaMissCount(), + getMetaHitCount() ); } catch (IllegalStateException e) { // Try again and let the exception bubble if it fails log.warn("getMetaData failed for read only store - attempting to reload!", e); return LookupMetadata.open( getMetadataBlobs(), - getMetaDataGeneration() + getMetaDataGeneration(), + getMetaMissCount(), + getMetaHitCount() ); } } else { try { return LookupMetadata.open( getMetadataBlobs(), - getMetaDataGeneration() + getMetaDataGeneration(), + getMetaMissCount(), + getMetaHitCount() ); } catch (IllegalStateException e) { log.warn("getMetaData failed for read write store - attempting to repair it!", e); @@ -366,7 +409,7 @@ private synchronized LookupMetadata repairMetadata() { int sortedPositionsSize = sortedPositions.length; LookupKey minKey = sortedPositionsSize > 0 ? readKey((long) sortedPositions[0]) : null; LookupKey maxKey = sortedPositionsSize > 0 ? readKey((long) sortedPositions[sortedPositionsSize - 1]) : null; - return LookupMetadata.generateMetadata(minKey, maxKey, sortedPositions, metadataBlobs, metaDataGeneration.incrementAndGet()); + return LookupMetadata.generateMetadata(minKey, maxKey, sortedPositions, metadataBlobs, metaDataGeneration.incrementAndGet(), getMetaMissCount(), getMetaHitCount()); } catch (IOException e) { throw new UncheckedIOException("Unable to write repaired metadata!", e); } @@ -376,6 +419,14 @@ private int getMetaDataGeneration() { return metaDataGeneration.get(); } + private LongAdder getMetaHitCount() { + return Optional.ofNullable(atomicMetadataRef.get()).map(md -> md.hitCount).orElse(new LongAdder()); + } + + private LongAdder getMetaMissCount() { + return Optional.ofNullable(atomicMetadataRef.get()).map(md -> md.missCount).orElse(new LongAdder()); + } + /** * Create a copy of the keys currently in the write cache * @@ -447,7 +498,6 @@ void flushWriteCache(LookupMetadata currentMetadata) { return null; }); } - ); } finally { @@ -455,7 +505,6 @@ void flushWriteCache(LookupMetadata currentMetadata) { } log.debug("flushed keys"); - } void generateMetaData(LookupMetadata currentMetadata) { @@ -463,6 +512,10 @@ void generateMetaData(LookupMetadata currentMetadata) { int flushSize = flushCache.size(); + // Increment the stats here + flushedKeyCounter += flushSize; + flushCounter ++; + // Update the counter and flush again if there are still more entries in the write cache than the threshold if (flushThreshold != -1 && writeCacheCounter.addAndGet(-flushSize) > flushThreshold) { AutoFlusher.submitWork(this::flush); @@ -503,73 +556,74 @@ void generateMetaData(LookupMetadata currentMetadata) { log.debug("Finished creating sortOrder"); try { - synchronized (flushReference) { - LookupMetadata metadata = LookupMetadata.generateMetadata(minKey, maxKey, newKeySortOrder, metadataBlobs, metaDataGeneration.incrementAndGet()); - partitionLookupCache.putMetadata(this, metadata); - } + atomicMetadataRef.set( + LookupMetadata + .generateMetadata( + minKey, + maxKey, + newKeySortOrder, + metadataBlobs, + metaDataGeneration.incrementAndGet(), + getMetaMissCount(), + getMetaHitCount()) + ); } catch (IOException e) { throw new UncheckedIOException("Failed to write new metadata!", e); } } - void flushCacheToReadCache() { - Iterator> iterator = flushCache.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - lookupBiConsumer.accept(entry.getKey(), entry.getValue()); - iterator.remove(); - } - } - - private LookupMetadata getMetadata() { + protected LookupMetadata getMetadata() { if (readOnly){ - return partitionLookupCache.getMetadata(this); - } else { + int[] stamp = new int[1]; + LookupMetadata result = timeStampedMetadata.get(stamp); - try { - metadataReadLock.lock(); - LookupMetadata metadata = flushReference.get(); - if (Objects.isNull(metadata)) { - metadata = partitionLookupCache.getMetadata(this); + // Convert millis to seconds + if (((System.currentTimeMillis() - startTime) / 1000) > stamp[0]){ + boolean luckyMe = reloadStamp.compareAndSet(stamp[0], stamp[0] + reloadInterval); + + if (luckyMe) { + result = loadMetadata(); + timeStampedMetadata.set(result, stamp[0] + reloadInterval); } - return metadata; - } finally { - metadataReadLock.unlock(); } + return result; + } else { + + return atomicMetadataRef.get(); } } - @Override public synchronized void flush() { if (readOnly) throw new RuntimeException("Can not flush read only LookupData"); if (writeCache.size() > 0) { + flushing.set(true); log.debug("starting flush"); - LookupMetadata currentMetadata; - try { - // Need to atomically get the cached metadata and hold the reference while we flush - metadataWriteLock.lock(); - currentMetadata = partitionLookupCache.getMetadata(this); - flushReference.set(currentMetadata); - } finally { - metadataWriteLock.unlock(); - } + LookupMetadata md = atomicMetadataRef.get(); + flushWriteCache(md); - try { - flushWriteCache(currentMetadata); + generateMetaData(md); - generateMetaData(currentMetadata); + flushCache.clear(); - // record stats about flushing - partitionLookupCache.addFlushCount(flushCache.size()); - } finally { - flushReference.set(null); - } - flushCacheToReadCache(); log.debug("flushed"); } + flushing.set(false); + } + + @Override + public void trim() { + if (!readOnly) { + flush(); + } else { + LookupMetadata result = timeStampedMetadata.get(new int[1]); + int stamp = (int) ((System.currentTimeMillis() - startTime) / 1000) - 1; + // set to elapsed time minus one - it will reload next time it is used. + timeStampedMetadata.set(result, stamp); + reloadStamp.set(stamp); + } } private int[] getKeyPosition() { @@ -654,5 +708,4 @@ public void scan(BiConsumer keyValueFunction) { .forEach(entry -> keyValueFunction .accept(entry.getKey(), entry.getValue())); } - } diff --git a/src/main/java/com/upserve/uppend/lookup/LookupMetadata.java b/src/main/java/com/upserve/uppend/lookup/LookupMetadata.java index 42c04981..22b9d6a7 100644 --- a/src/main/java/com/upserve/uppend/lookup/LookupMetadata.java +++ b/src/main/java/com/upserve/uppend/lookup/LookupMetadata.java @@ -6,12 +6,34 @@ import java.io.IOException; import java.lang.invoke.MethodHandles; import java.nio.*; -import java.util.concurrent.ConcurrentHashMap; - +import java.util.Arrays; +import java.util.concurrent.atomic.LongAdder; +/** + * The bisect tree is linearized as follows + * 8 + * 4 + * 9 + * 2 + * 10 + * 5 + * 11 + * 1 + * 12 + * 6 + * 13 + * 3 + * 14 + * 7 + * 15 + * The size of the array containing the tree is 2^(n+1) + * If n is the current index the branch above is 2*n and the branch below is 2*n+1 + */ public class LookupMetadata { private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private static final int MAX_BISECT_KEY_CACHE_DEPTH = 9; + private static final int MAX_BISECT_KEY_CACHE_DEPTH = 9; // Size == 1024 + private static final int MAX_TREE_NODES = treeSize(MAX_BISECT_KEY_CACHE_DEPTH); + private final LookupKey[] bisectKeys = new LookupKey[MAX_TREE_NODES]; private final int metadataGeneration; @@ -20,15 +42,17 @@ public class LookupMetadata { private final LookupKey maxKey; private final int[] keyStorageOrder; - private final ConcurrentHashMap bisectKeys; - - public static LookupMetadata generateMetadata(LookupKey minKey, LookupKey maxKey, int[] keyStorageOrder, VirtualMutableBlobStore metaDataBlobs, int metadataGeneration) throws IOException { + final LongAdder hitCount; + final LongAdder missCount; + public static LookupMetadata generateMetadata(LookupKey minKey, LookupKey maxKey, int[] keyStorageOrder, VirtualMutableBlobStore metaDataBlobs, int metadataGeneration, LongAdder missCount, LongAdder hitCount) throws IOException { LookupMetadata newMetadata = new LookupMetadata( minKey, maxKey, keyStorageOrder, - metadataGeneration + metadataGeneration, + missCount, + hitCount ); newMetadata.writeTo(metaDataBlobs); @@ -37,25 +61,36 @@ public static LookupMetadata generateMetadata(LookupKey minKey, LookupKey maxKey } LookupMetadata(LookupKey minKey, LookupKey maxKey, int[] keyStorageOrder, int metadataGeneration) { + this(minKey, maxKey, keyStorageOrder, metadataGeneration, new LongAdder(), new LongAdder()); + } + + private LookupMetadata(LookupKey minKey, LookupKey maxKey, int[] keyStorageOrder, int metadataGeneration, LongAdder missCount, LongAdder hitCount) { this.numKeys = keyStorageOrder.length; this.minKey = minKey; this.maxKey = maxKey; this.keyStorageOrder = keyStorageOrder; this.metadataGeneration = metadataGeneration; - bisectKeys = new ConcurrentHashMap<>(); + this.hitCount = hitCount; + this.missCount = missCount; + } public static LookupMetadata open(VirtualMutableBlobStore metadataBlobs, int metadataGeneration) { + return open(metadataBlobs, metadataGeneration, new LongAdder(), new LongAdder()); + } + + public static LookupMetadata open(VirtualMutableBlobStore metadataBlobs, int metadataGeneration, LongAdder missCount, LongAdder hitCount) { + // TODO can we preserve bisectKeys if the content is not changed? (Currently periodic reload clears this cache) if (metadataBlobs.isPageAllocated(0L)) { byte[] bytes = metadataBlobs.read(0L); - return new LookupMetadata(bytes, metadataGeneration); + return new LookupMetadata(bytes, metadataGeneration, missCount, hitCount); } else { - return new LookupMetadata(null, null, new int[0], metadataGeneration); + return new LookupMetadata(null, null, new int[0], metadataGeneration, missCount, hitCount); } } - LookupMetadata(byte[] bytes, int metadataGeneration) { + private LookupMetadata(byte[] bytes, int metadataGeneration, LongAdder missCount, LongAdder hitCount) { ByteBuffer buffer = ByteBuffer.wrap(bytes); int minKeyLength, maxKeyLength; @@ -79,7 +114,8 @@ public static LookupMetadata open(VirtualMutableBlobStore metadataBlobs, int met this.metadataGeneration = metadataGeneration; - bisectKeys = new ConcurrentHashMap<>(); + this.hitCount = hitCount; + this.missCount = missCount; } /** @@ -92,13 +128,13 @@ public static LookupMetadata open(VirtualMutableBlobStore metadataBlobs, int met * @param key the key to find and mark * @return the position of the key */ - public Long findKey(VirtualLongBlobStore longBlobStore, LookupKey key) { key.setMetaDataGeneration(metadataGeneration); if (numKeys == 0) { key.setInsertAfterSortIndex(-1); + missCount.increment(); return null; } @@ -108,6 +144,8 @@ public Long findKey(VirtualLongBlobStore longBlobStore, LookupKey key) { LookupKey upperKey = maxKey; int bisectCount = 0; + int bisectKeyTreeArrayIndex = 1; + int keyPosition; LookupKey midpointKey; int midpointKeyIndex; @@ -115,25 +153,30 @@ public Long findKey(VirtualLongBlobStore longBlobStore, LookupKey key) { int comparison = lowerKey.compareTo(key); if (comparison > 0 /* new key is less than lowerKey */) { key.setInsertAfterSortIndex(-1); // Insert it after this index in the sort order + missCount.increment(); return null; } if (comparison == 0) { key.setPosition(keyStorageOrder[keyIndexLower]); + hitCount.increment(); return longBlobStore.readLong(keyStorageOrder[keyIndexLower]); } comparison = upperKey.compareTo(key); if (comparison < 0 /* new key is greater than upperKey */) { key.setInsertAfterSortIndex(keyIndexUpper); // Insert it after this index in the sort order + missCount.increment(); return null; } if (comparison == 0) { key.setPosition(keyStorageOrder[keyIndexUpper]); + hitCount.increment(); return longBlobStore.readLong(keyStorageOrder[keyIndexUpper]); } if (numKeys == 2) { // There are no other values keys besides upper and lower key.setInsertAfterSortIndex(keyIndexLower); + missCount.increment(); return null; } @@ -147,30 +190,50 @@ public Long findKey(VirtualLongBlobStore longBlobStore, LookupKey key) { keyPosition = keyStorageOrder[midpointKeyIndex]; // Cache only the most frequently used midpoint keys if (bisectCount < MAX_BISECT_KEY_CACHE_DEPTH) { - midpointKey = bisectKeys.computeIfAbsent(keyPosition, position -> new LookupKey(longBlobStore.readBlob(position))); + if (bisectKeys[bisectKeyTreeArrayIndex] == null){ + midpointKey = bisectKeys[bisectKeyTreeArrayIndex] = new LookupKey(longBlobStore.readBlob(keyPosition)); + } else { + midpointKey = bisectKeys[bisectKeyTreeArrayIndex]; + } } else { midpointKey = new LookupKey(longBlobStore.readBlob(keyPosition)); } comparison = key.compareTo(midpointKey); + + if (comparison == 0) { + key.setPosition(keyPosition); + hitCount.increment(); + return longBlobStore.readLong(keyPosition); + } + if (comparison < 0) { upperKey = midpointKey; keyIndexUpper = midpointKeyIndex; - } else if (comparison > 0) { - keyIndexLower = midpointKeyIndex; - lowerKey = midpointKey; + bisectKeyTreeArrayIndex = bisectKeyTreeArrayIndex * 2; + } else { - key.setPosition(keyPosition); - return longBlobStore.readLong(keyPosition); + lowerKey = midpointKey; + keyIndexLower = midpointKeyIndex; + bisectKeyTreeArrayIndex = bisectKeyTreeArrayIndex * 2 + 1; } bisectCount++; } while ((keyIndexLower + 1) < keyIndexUpper); key.setInsertAfterSortIndex(keyIndexLower); // Insert it in the sort order after this key + missCount.increment(); return null; } + public static int treeSize(int depth) { + return 1 << (depth +1); + } + + void clearLookupTree(){ + Arrays.fill(bisectKeys, null); + } + public void writeTo(VirtualMutableBlobStore metadataBlobs) { int headerSize = 12 + minKey.byteLength() + maxKey.byteLength(); int intBufSize = 4 * numKeys; @@ -213,6 +276,14 @@ public int getNumKeys() { return numKeys; } + public long getHitCount() { + return hitCount.sum(); + } + + public long getMissCount(){ + return missCount.sum(); + } + public int[] getKeyStorageOrder() { return keyStorageOrder; } diff --git a/src/main/java/com/upserve/uppend/lookup/PartitionLookupCache.java b/src/main/java/com/upserve/uppend/lookup/PartitionLookupCache.java deleted file mode 100644 index 3d49d35a..00000000 --- a/src/main/java/com/upserve/uppend/lookup/PartitionLookupCache.java +++ /dev/null @@ -1,50 +0,0 @@ -package com.upserve.uppend.lookup; - -import java.util.function.Function; - -/** - * AppendStorePartition level view of the store lookup cache - */ -public class PartitionLookupCache { - - private final LookupCache lookupCache; - private final String partition; - - - public static PartitionLookupCache create(String partition, LookupCache lookupCache) { - return new PartitionLookupCache(partition.intern(), lookupCache); - } - - public PartitionLookupCache(String partition, LookupCache lookupCache) { - this.partition = partition; - this.lookupCache = lookupCache; - } - - public String getPartition() { - return partition; - } - - public void putLookup(LookupKey key, long val) { - lookupCache.putLookup(new PartitionLookupKey(partition, key), val); - } - - public Long getLong(LookupKey lookupKey, Function cacheLoader) { - return lookupCache.getLong(new PartitionLookupKey(partition, lookupKey), cacheLoader); - } - - public boolean isKeyCacheActive(){ - return lookupCache.isKeyCacheActive(); - } - - public LookupMetadata getMetadata(LookupData key) { - return lookupCache.getMetadata(key); - } - - public void putMetadata(LookupData key, LookupMetadata value) { - lookupCache.putMetadata(key, value); - } - - public void addFlushCount(long val){ - lookupCache.addFlushCount(val); - } -} diff --git a/src/main/java/com/upserve/uppend/lookup/PartitionLookupKey.java b/src/main/java/com/upserve/uppend/lookup/PartitionLookupKey.java deleted file mode 100644 index dbab4723..00000000 --- a/src/main/java/com/upserve/uppend/lookup/PartitionLookupKey.java +++ /dev/null @@ -1,35 +0,0 @@ -package com.upserve.uppend.lookup; - -import java.util.Objects; - -public class PartitionLookupKey { - private final LookupKey lookupKey; - private final String partition; - - public PartitionLookupKey(String partition, LookupKey lookupKey) { - this.partition = partition; - this.lookupKey = lookupKey; - } - - public LookupKey getLookupKey() { - return lookupKey; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - PartitionLookupKey that = (PartitionLookupKey) o; - return Objects.equals(lookupKey, that.lookupKey) && - Objects.equals(partition, that.partition); - } - - @Override - public int hashCode() { - return Objects.hash(lookupKey, partition); - } - - public int weight() { - return lookupKey.byteLength(); - } -} diff --git a/src/main/java/com/upserve/uppend/metrics/AppendOnlyStoreWithMetrics.java b/src/main/java/com/upserve/uppend/metrics/AppendOnlyStoreWithMetrics.java index e546e58f..1ae5a3d7 100644 --- a/src/main/java/com/upserve/uppend/metrics/AppendOnlyStoreWithMetrics.java +++ b/src/main/java/com/upserve/uppend/metrics/AppendOnlyStoreWithMetrics.java @@ -1,10 +1,8 @@ package com.upserve.uppend.metrics; import com.codahale.metrics.*; -import com.github.benmanes.caffeine.cache.stats.CacheStats; import com.google.common.collect.Maps; import com.upserve.uppend.*; -import com.upserve.uppend.lookup.FlushStats; import java.util.Map; import java.util.function.BiConsumer; @@ -186,33 +184,13 @@ public String getName() { } @Override - public FlushStats getFlushStats() { - return store.getFlushStats(); - } - - @Override - public CacheStats getBlobPageCacheStats() { - return store.getBlobPageCacheStats(); - } - - @Override - public CacheStats getKeyPageCacheStats() { - return store.getKeyPageCacheStats(); - } - - @Override - public CacheStats getLookupKeyCacheStats() { - return store.getLookupKeyCacheStats(); - } - - @Override - public CacheStats getMetadataCacheStats() { - return store.getMetadataCacheStats(); + public BlockStats getBlockLongStats() { + return store.getBlockLongStats(); } @Override - public BlockStats getBlockLongStats() { - return store.getBlockLongStats(); + public PartitionStats getPartitionStats() { + return store.getPartitionStats(); } @Override diff --git a/src/main/java/com/upserve/uppend/metrics/CounterStoreWithMetrics.java b/src/main/java/com/upserve/uppend/metrics/CounterStoreWithMetrics.java index 2c3ed55b..506cbc9d 100644 --- a/src/main/java/com/upserve/uppend/metrics/CounterStoreWithMetrics.java +++ b/src/main/java/com/upserve/uppend/metrics/CounterStoreWithMetrics.java @@ -1,9 +1,7 @@ package com.upserve.uppend.metrics; import com.codahale.metrics.*; -import com.github.benmanes.caffeine.cache.stats.CacheStats; import com.upserve.uppend.CounterStore; -import com.upserve.uppend.lookup.FlushStats; import java.util.Map; import java.util.function.ObjLongConsumer; @@ -130,26 +128,6 @@ public void scan(ObjLongConsumer callback) { } } - @Override - public FlushStats getFlushStats() { - return store.getFlushStats(); - } - - @Override - public CacheStats getKeyPageCacheStats() { - return store.getKeyPageCacheStats(); - } - - @Override - public CacheStats getLookupKeyCacheStats() { - return store.getLookupKeyCacheStats(); - } - - @Override - public CacheStats getMetadataCacheStats() { - return store.getMetadataCacheStats(); - } - @Override public long keyCount() { return store.keyCount(); diff --git a/src/main/java/com/upserve/uppend/metrics/MetricsStatsCounter.java b/src/main/java/com/upserve/uppend/metrics/MetricsStatsCounter.java deleted file mode 100644 index c92bf47b..00000000 --- a/src/main/java/com/upserve/uppend/metrics/MetricsStatsCounter.java +++ /dev/null @@ -1,107 +0,0 @@ -package com.upserve.uppend.metrics; -/* - * Copyright 2016 Ben Manes. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import com.codahale.metrics.*; -import com.github.benmanes.caffeine.cache.stats.*; - -import java.util.concurrent.TimeUnit; - -import static java.util.Objects.requireNonNull; - -/** - * A {@link StatsCounter} instrumented with Dropwizard Metrics. - * - * @author ben.manes@gmail.com (Ben Manes) - */ -public final class MetricsStatsCounter implements StatsCounter { - private final Meter hitCount; - private final Meter missCount; - private final Meter loadSuccessCount; - private final Meter loadFailureCount; - private final Timer totalLoadTime; - private final Meter evictionCount; - private final Meter evictionWeight; - - /** - * Constructs an instance for use by a single cache. - * - * @param registry the registry of metric instances - * @param metricsPrefix the prefix name for the metrics - */ - public MetricsStatsCounter(MetricRegistry registry, String metricsPrefix) { - requireNonNull(metricsPrefix); - hitCount = registry.meter(metricsPrefix + ".hits"); - missCount = registry.meter(metricsPrefix + ".misses"); - totalLoadTime = registry.timer(metricsPrefix + ".loads"); - loadSuccessCount = registry.meter(metricsPrefix + ".loads-success"); - loadFailureCount = registry.meter(metricsPrefix + ".loads-failure"); - evictionCount = registry.meter(metricsPrefix + ".evictions"); - evictionWeight = registry.meter(metricsPrefix + ".evictions-weight"); - } - - @Override - public void recordHits(int count) { - hitCount.mark(count); - } - - @Override - public void recordMisses(int count) { - missCount.mark(count); - } - - @Override - public void recordLoadSuccess(long loadTime) { - loadSuccessCount.mark(); - totalLoadTime.update(loadTime, TimeUnit.NANOSECONDS); - } - - @Override - public void recordLoadFailure(long loadTime) { - loadFailureCount.mark(); - totalLoadTime.update(loadTime, TimeUnit.NANOSECONDS); - } - - @Override - @Deprecated - public void recordEviction() { - // This method is scheduled for removal in version 3.0 in favor of recordEviction(weight) - recordEviction(1); - } - - @Override - public void recordEviction(int weight) { - evictionCount.mark(); - evictionWeight.mark(weight); - } - - @Override - public CacheStats snapshot() { - return new CacheStats( - hitCount.getCount(), - missCount.getCount(), - loadSuccessCount.getCount(), - loadFailureCount.getCount(), - totalLoadTime.getCount(), - evictionCount.getCount(), - evictionWeight.getCount()); - } - - @Override - public String toString() { - return snapshot().toString(); - } -} \ No newline at end of file diff --git a/src/test/java/com/upserve/uppend/AppendOnlyStorePartitionSizeTest.java b/src/test/java/com/upserve/uppend/AppendOnlyStorePartitionSizeTest.java index 2e3906e1..ae56c88b 100644 --- a/src/test/java/com/upserve/uppend/AppendOnlyStorePartitionSizeTest.java +++ b/src/test/java/com/upserve/uppend/AppendOnlyStorePartitionSizeTest.java @@ -6,7 +6,6 @@ import java.io.IOException; import java.nio.file.*; -import java.util.stream.Stream; import static org.junit.Assert.assertEquals; @@ -14,7 +13,7 @@ public class AppendOnlyStorePartitionSizeTest { private final Path path = Paths.get("build/test/file-append-only-partition-size"); private AppendOnlyStore newStore() { - return TestHelper.getDefaultTestBuilder().withDir(path.resolve("store-path")).withPartitionSize(2).build(false); + return TestHelper.getDefaultAppendStoreTestBuilder().withDir(path.resolve("store-path")).withPartitionCount(2).build(false); } private AppendOnlyStore store; @@ -28,7 +27,7 @@ public void initialize() throws IOException { } @After - public void cleanUp() throws IOException { + public void cleanUp() { try { store.close(); } catch (Exception e) { diff --git a/src/test/java/com/upserve/uppend/AppendOnlyStoreTest.java b/src/test/java/com/upserve/uppend/AppendOnlyStoreTest.java index 11d6b95e..cb0992ed 100644 --- a/src/test/java/com/upserve/uppend/AppendOnlyStoreTest.java +++ b/src/test/java/com/upserve/uppend/AppendOnlyStoreTest.java @@ -13,6 +13,7 @@ import java.util.concurrent.*; import java.util.stream.Collectors; +import static com.upserve.uppend.TestHelper.genBytes; import static org.junit.Assert.*; public class AppendOnlyStoreTest { @@ -23,7 +24,7 @@ private AppendOnlyStore newStore() { } private AppendOnlyStore newStore(boolean readOnly) { - return TestHelper.getDefaultTestBuilder().withDir(path.resolve("store-path")).build(readOnly); + return TestHelper.getDefaultAppendStoreTestBuilder().withDir(path.resolve("store-path")).build(readOnly); } private AppendOnlyStore store; @@ -84,7 +85,7 @@ public void testEmptyReadOnlyStore() throws Exception { } @Test - public void testClear() throws Exception { + public void testClear() { String key = "foobar"; byte[] bytes = genBytes(12); @@ -99,7 +100,7 @@ public void testClear() throws Exception { } @Test - public void testPurge() throws Exception { + public void testTrim() { String key = "foobar"; byte[] bytes = genBytes(12); @@ -117,7 +118,7 @@ public void testAppendWhileFlushing() throws Exception { ExecutorService executor = new ForkJoinPool(); Future future = executor.submit(() -> { new Random(314159) - .longs(100_000, 0, 5_000) + .longs(500_000, 0, 5_000) .parallel() .forEach(val -> { String key = String.valueOf(val); @@ -149,6 +150,8 @@ public void testAppendWhileFlushing() throws Exception { }); future.get(40_000, TimeUnit.MILLISECONDS); + + executor.shutdown(); } @Test @@ -180,7 +183,7 @@ public void testWriteThenClearThenClose() throws Exception { @Test public void fillTheCache() throws Exception { - int keys = TestHelper.getDefaultTestBuilder().getInitialLookupKeyCacheSize() * 2; + int keys = 1024 * 256; Random random = new Random(9876); Set uuidSet = new HashSet<>(); @@ -387,37 +390,11 @@ private void tester(int number, int size) { assertEquals(inputBytes.size(), outputBytes.size()); - inputBytes.sort(AppendOnlyStoreTest::compareByteArrays); - outputBytes.sort(AppendOnlyStoreTest::compareByteArrays); + inputBytes.sort(TestHelper::compareByteArrays); + outputBytes.sort(TestHelper::compareByteArrays); for (int i = 0; i < number; i++) { assertArrayEquals("input and output byte arrays differ at index " + i, inputBytes.get(i), outputBytes.get(i)); } } - - private static int compareByteArrays(byte[] o1, byte[] o2) { - if (o1 == null) { - if (o2 == null) { - return 0; - } - return -1; - } - if (o2 == null) { - return 1; - } - for (int i = 0; i < o1.length && i < o2.length; i++) { - int v1 = 0xff & o1[i]; - int v2 = 0xff & o2[i]; - if (v1 != v2) { - return v1 < v2 ? -1 : 1; - } - } - return Integer.compare(o1.length, o2.length); - } - - private byte[] genBytes(int len) { - byte[] bytes = new byte[len]; - new Random().nextBytes(bytes); - return bytes; - } } diff --git a/src/test/java/com/upserve/uppend/CounterStoreTest.java b/src/test/java/com/upserve/uppend/CounterStoreTest.java index b4731c19..9a1ccc35 100644 --- a/src/test/java/com/upserve/uppend/CounterStoreTest.java +++ b/src/test/java/com/upserve/uppend/CounterStoreTest.java @@ -24,7 +24,7 @@ private CounterStore newStore() { return newStore(false); } private CounterStore newStore(boolean readOnly) { - return CounterStoreBuilder.getDefaultTestBuilder().withDir(path.resolve("store-path")).build(readOnly); + return TestHelper.getDefaultCounterStoreTestBuilder().withDir(path.resolve("store-path")).build(readOnly); } private CounterStore store; @@ -94,7 +94,6 @@ public void incrTwiceTwoTest() throws Exception { assertEquals(Long.valueOf(3), store.get("partition", "foo")); } - @Test public void testWriteCloseReadRepeat() throws Exception { store.set("partition", "foo", 5); @@ -208,9 +207,7 @@ public void testExample() { store.increment("2017-11-30", "bbbbbbbb-bbbbbbb-bbbb-bbbbbbb-bbbb::bbbbbbb"); store.increment("2017-11-30", "bbbbbbbb-bbbbbbb-bbbb-bbbbbbb-bbbb::bbbbbbb"); store.increment("2017-11-30", "bbbbbbbb-bbbbbbb-bbbb-bbbbbbb-bbbb::bbbbbbb"); - store.increment("2017-11-30", "ccccccc-cccccccccc-ccccccc-ccccccc::ccccccc"); - store.increment("2017-11-30", "ttt-ttttt-tttt-ttttttt-ttt-tttt::tttttttttt"); assertEquals(Long.valueOf(5), store.get("2017-11-30", "bbbbbbbb-bbbbbbb-bbbb-bbbbbbb-bbbb::bbbbbbb")); diff --git a/src/test/java/com/upserve/uppend/FileStoreTest.java b/src/test/java/com/upserve/uppend/FileStoreTest.java index e21fc591..524b955c 100644 --- a/src/test/java/com/upserve/uppend/FileStoreTest.java +++ b/src/test/java/com/upserve/uppend/FileStoreTest.java @@ -4,8 +4,10 @@ import org.junit.*; import java.io.*; +import java.nio.ByteBuffer; import java.nio.file.*; -import java.util.Optional; +import java.util.*; +import java.util.stream.Collectors; import static org.junit.Assert.*; @@ -97,6 +99,23 @@ public void testPartitionHashExamplesWhenNotHashed() { assertEquals("9", v.partitionHash("9")); } + @Test + public void testCreateCloseOpenReadOnlyClose() { + MyFileStore store = new MyFileStore(path.resolve("create-close-open-close"), 0); + store.getOrCreate("p1").append("k1", "v1".getBytes()); + store.close(); + + store = new MyFileStore(path.resolve("create-close-open-close"), 0, true); + byte[][] result = store.getIfPresent("p1") + .map(partition -> partition.read("k1")) + .map(byteStream -> byteStream.toArray(byte[][]::new)) + .orElse(new byte[][]{}); + byte[][] expected = new byte[][]{"v1".getBytes()}; + assertArrayEquals(expected, result); + + store.close(); + } + @Test public void testGetIfPresent() { MyFileStore v = new MyFileStore(path.resolve("get-if-present"), 0); @@ -178,11 +197,57 @@ public void testCreateDirectoriesWithParentSymlink() throws Exception { SafeDeleting.removeDirectory(base); } + @Test + public void testReaderWriter() throws InterruptedException { + + MyFileStore reader = new MyFileStore(path.resolve("reader_writer"), 4, true); + assertEquals(0, reader.read("foo","bar").count()); + assertEquals(0, reader.scan().flatMap(Map.Entry::getValue).count()); + + MyFileStore writer = new MyFileStore(path.resolve("reader_writer"), 4, false); + writer.append("foo", "bar", "abc".getBytes()); + writer.flush(); + + assertEquals( + Collections.singletonList(ByteBuffer.wrap("abc".getBytes())), + reader.read("foo","bar").map(ByteBuffer::wrap).collect(Collectors.toList()) + ); + assertEquals( + Collections.singletonList(ByteBuffer.wrap("abc".getBytes())), + reader.scan().flatMap(Map.Entry::getValue).map(ByteBuffer::wrap).collect(Collectors.toList()) + ); + + writer.append("foo", "baz", "def".getBytes()); + writer.flush(); + + assertEquals(0, reader.read("foo","baz").count()); + assertEquals( + Collections.singletonList(ByteBuffer.wrap("abc".getBytes())), + reader.scan().flatMap(Map.Entry::getValue).map(ByteBuffer::wrap).collect(Collectors.toList()) + ); + + reader.trim(); + + assertEquals( + Collections.singletonList(ByteBuffer.wrap("def".getBytes())), + reader.read("foo","baz").map(ByteBuffer::wrap).collect(Collectors.toList()) + ); + assertEquals( + List.of(ByteBuffer.wrap("abc".getBytes()), ByteBuffer.wrap("def".getBytes())), + reader.scan().flatMap(Map.Entry::getValue).map(ByteBuffer::wrap).sorted().collect(Collectors.toList()) + ); + } + private class MyFileStore extends FileAppendOnlyStore { MyFileStore(Path dir, int numPartitions) { - super(false, new AppendOnlyStoreBuilder() + this(dir, numPartitions, false); + } + + MyFileStore(Path dir, int numPartitions, boolean readOnly) { + super(readOnly, new AppendOnlyStoreBuilder() .withDir(dir) - .withPartitionSize(numPartitions) + .withPartitionCount(numPartitions) + .withMetadataTTL(30) ); } } diff --git a/src/test/java/com/upserve/uppend/TestHelper.java b/src/test/java/com/upserve/uppend/TestHelper.java index 07110bef..e2a5836c 100644 --- a/src/test/java/com/upserve/uppend/TestHelper.java +++ b/src/test/java/com/upserve/uppend/TestHelper.java @@ -1,16 +1,78 @@ package com.upserve.uppend; -import org.slf4j.*; - +import org.junit.*; +import org.slf4j.LoggerFactory; +import java.io.*; import java.lang.reflect.*; import java.util.concurrent.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + public class TestHelper { + + public static class IoStreamHelper { + private static final PrintStream stdout = System.out; + private static final PrintStream stderr = System.err; + + private final ByteArrayOutputStream outStream = new ByteArrayOutputStream(); + private final ByteArrayOutputStream errStream = new ByteArrayOutputStream(); + + String getOutString() { + System.out.flush(); + return outStream.toString(); + } + + String getErrString() { + System.err.flush(); + return errStream.toString(); + } + + protected void assertStdErrContains(String expected) { + String errStr = getErrString(); + assertTrue("didn't find expected '" + expected + "' in captured stderr: \n" + errStr, errStr.contains(expected)); + } + + protected void assertStdOutContains(String expected) { + String outString = getOutString(); + assertTrue("didn't find expected '" + expected + "' in captured stdout: \n" + outString, outString.contains(expected)); + } + + protected void assertStdOut(String expected) { + assertEquals("Captured standard output string not equal", expected, getOutString()); + } + + protected void assertStdErr(String expected) { + assertEquals("Captured standard error string not equal", expected, getErrString()); + } + + protected void resetStreams() { + System.out.flush(); + outStream.reset(); + System.err.flush(); + errStream.reset(); + } + + @Before + public void setUpStreams() { + outStream.reset(); + System.setOut(new PrintStream(outStream)); + errStream.reset(); + System.setErr(new PrintStream(errStream)); + } + + @After + public void cleanUpStreams() { + System.setOut(stdout); + System.setErr(stderr); + } + } + public static void resetLogger(Class clazz, String fieldName) throws Exception { setLogger(clazz, fieldName, LoggerFactory.getLogger(clazz)); } - public static void setLogger(Class clazz, String fieldName, Logger log) throws Exception { + public static void setLogger(Class clazz, String fieldName, org.slf4j.Logger log) throws Exception { Field field = clazz.getDeclaredField(fieldName); field.setAccessible(true); Field modifiersField = Field.class.getDeclaredField("modifiers"); @@ -19,29 +81,50 @@ public static void setLogger(Class clazz, String fieldName, Logger log) throws E field.set(null, log); } - public static AppendOnlyStoreBuilder getDefaultTestBuilder() { - return getDefaultTestBuilder(ForkJoinPool.commonPool()); + public static AppendOnlyStoreBuilder getDefaultAppendStoreTestBuilder() { + return getDefaultAppendStoreTestBuilder(ForkJoinPool.commonPool()); } - public static AppendOnlyStoreBuilder getDefaultTestBuilder(ExecutorService testService) { + public static AppendOnlyStoreBuilder getDefaultAppendStoreTestBuilder(ExecutorService testService) { return new AppendOnlyStoreBuilder() .withStoreName("test") .withBlobPageSize(64 * 1024) .withBlobsPerBlock(30) - .withInitialBlobCacheSize(64) - .withMaximumBlobCacheSize(128) - .withBlobCacheExecutorService(testService) - .withInitialLookupKeyCacheSize(64) - .withMaximumLookupKeyCacheWeight(100 * 1024) - .withLookupKeyCacheExecutorService(testService) - .withInitialMetaDataCacheSize(64) - .withMaximumMetaDataCacheWeight(100 * 1024) - .withLookupMetaDataCacheExecutorService(testService) - .withLongLookupHashSize(16) + .withTargetBufferSize(16*1024*1024) + .withLongLookupHashCount(16) .withLookupPageSize(16 * 1024) - .withLookupPageCacheExecutorService(testService) - .withCacheMetrics(); + .withMetadataTTL(0); + } + public static CounterStoreBuilder getDefaultCounterStoreTestBuilder() { + return new CounterStoreBuilder() + .withStoreName("test") + .withTargetBufferSize(16*1024*1024) + .withMetadataPageSize(1024) + .withLongLookupHashCount(16) + .withLookupPageSize(16 * 1024); } + public static int compareByteArrays(byte[] o1, byte[] o2) { + if (o1 == null) { + return o2 == null ? 0 : -1; + } + if (o2 == null) { + return 1; + } + for (int i = 0; i < o1.length && i < o2.length; i++) { + int v1 = 0xff & o1[i]; + int v2 = 0xff & o2[i]; + if (v1 != v2) { + return v1 < v2 ? -1 : 1; + } + } + return Integer.compare(o1.length, o2.length); + } + + public static byte[] genBytes(int len) { + byte[] bytes = new byte[len]; + ThreadLocalRandom.current().nextBytes(bytes); + return bytes; + } } diff --git a/src/test/java/com/upserve/uppend/UppendMainTest.java b/src/test/java/com/upserve/uppend/UppendMainTest.java index 33e1c82c..a7b7da78 100644 --- a/src/test/java/com/upserve/uppend/UppendMainTest.java +++ b/src/test/java/com/upserve/uppend/UppendMainTest.java @@ -2,30 +2,11 @@ import org.junit.*; -import java.io.*; - -import static org.junit.Assert.assertTrue; - -public class UppendMainTest { - private final ByteArrayOutputStream outStream = new ByteArrayOutputStream(); - private final ByteArrayOutputStream errStream = new ByteArrayOutputStream(); - - @Before - public void setUpStreams() { - System.setOut(new PrintStream(outStream)); - System.setErr(new PrintStream(errStream)); - } - - @After - public void cleanUpStreams() { - System.setOut(null); - System.setErr(null); - } - +public class UppendMainTest extends TestHelper.IoStreamHelper { @Test public void testMain() throws Exception { - Uppend.main("--help"); - String outStr = outStream.toString(); - assertTrue("didn't find expected 'Usage: uppend' in main stderr output: " + outStr, outStr.contains("Usage: uppend")); + Uppend.main(); + assertStdErrContains("Usage: uppend"); + assertStdOut(""); } } diff --git a/src/test/java/com/upserve/uppend/blobs/FilePageTest.java b/src/test/java/com/upserve/uppend/blobs/FilePageTest.java new file mode 100644 index 00000000..4761556f --- /dev/null +++ b/src/test/java/com/upserve/uppend/blobs/FilePageTest.java @@ -0,0 +1,68 @@ +package com.upserve.uppend.blobs; + +import org.junit.*; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.io.*; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class FilePageTest { + + @Mock + FileChannel channel; + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + FilePage filePage; + + @Before + public void before() { + filePage = new FilePage(channel, 0, 1024); + } + + @Test + public void getIOExceptionTest() throws IOException { + when(channel.read(any(ByteBuffer.class), anyLong())).thenThrow(IOException.class); + thrown.expect(UncheckedIOException.class); + filePage.get(0,new byte[100], 0); + } + @Test + public void putIOExceptionTest() throws IOException { + when(channel.write(any(ByteBuffer.class), anyLong())).thenThrow(IOException.class); + thrown.expect(UncheckedIOException.class); + filePage.put(0,new byte[100], 0); + } + @Test + public void getIllegalStateText() throws IOException{ + when(channel.read(any(ByteBuffer.class), anyLong())).thenReturn(127); + thrown.expect(IllegalStateException.class); + filePage.get(0,new byte[130], 7); + + } + @Test + public void putIllegalStateText() throws IOException{ + when(channel.write(any(ByteBuffer.class), anyLong())).thenReturn(127); + thrown.expect(IllegalStateException.class); + filePage.put(0,new byte[130], 7); + } + @Test + public void getTest() throws IOException{ + when(channel.read(any(ByteBuffer.class), anyLong())).thenReturn(123); + filePage.get(0,new byte[130], 7); + } + @Test + public void putTest() throws IOException{ + when(channel.write(any(ByteBuffer.class), anyLong())).thenReturn(123);; + filePage.put(0,new byte[130], 7); + } +} diff --git a/src/test/java/com/upserve/uppend/blobs/MappedPageTest.java b/src/test/java/com/upserve/uppend/blobs/MappedPageTest.java index 121e3679..ace22ce2 100644 --- a/src/test/java/com/upserve/uppend/blobs/MappedPageTest.java +++ b/src/test/java/com/upserve/uppend/blobs/MappedPageTest.java @@ -15,7 +15,7 @@ import static org.junit.Assert.assertEquals; public class MappedPageTest { - Path rootPath = Paths.get("build/test/blobs/filepage"); + Path rootPath = Paths.get("build/test/blobs/mapped_page"); Path filePath = rootPath.resolve("testfile"); Path readOnlyFilePath = rootPath.resolve("readOnlyTestfile"); diff --git a/src/test/java/com/upserve/uppend/blobs/PageCacheTest.java b/src/test/java/com/upserve/uppend/blobs/PageCacheTest.java deleted file mode 100644 index 8d4d920f..00000000 --- a/src/test/java/com/upserve/uppend/blobs/PageCacheTest.java +++ /dev/null @@ -1,89 +0,0 @@ -package com.upserve.uppend.blobs; - -import com.upserve.uppend.*; -import com.upserve.uppend.util.SafeDeleting; -import org.junit.*; -import org.junit.rules.ExpectedException; - -import java.io.IOException; -import java.nio.file.*; -import java.util.concurrent.*; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; - -public class PageCacheTest { - private final String name = "page_cache_test"; - Path rootPath = Paths.get("build/test/blobs").resolve(name); - Path existingFile = rootPath.resolve("existing_file"); - Path fileDoesNotExist = rootPath.resolve("file_does_not_exist"); - Path pathDoesNotExist = rootPath.resolve("path_does_not_exist/file"); - - ExecutorService testService = new ForkJoinPool(); - private AppendOnlyStoreBuilder defaults; - - VirtualPageFile virtualPageFile; - PageCache instance; - - @Rule - public ExpectedException thrown = ExpectedException.none(); - - @Before - public void before() throws IOException { - SafeDeleting.removeDirectory(rootPath); - Files.createDirectories(rootPath); - Files.createFile(existingFile); - } - - private void setup(boolean readOnly) { - testService = new ForkJoinPool(); - - defaults = TestHelper - .getDefaultTestBuilder(testService) - .withBlobPageSize(512) - .withInitialBlobCacheSize(128) - .withMaximumBlobCacheSize(512); - - instance = defaults.buildBlobPageCache(name); - - virtualPageFile = new VirtualPageFile(existingFile, 12, false, instance); - } - - @After - public void shutdown() throws InterruptedException { - if (instance != null) { - instance.flush(); - } - - if (testService != null) { - testService.shutdown(); - testService.awaitTermination(1000, TimeUnit.MILLISECONDS); - } - } - - @Test - public void testGetPageFlush() { - setup(false); - - final long position = 1284; - Page page; - - page = instance.get(position, virtualPageFile.getFilePath(), pageKey -> virtualPageFile.mappedPage(pageKey.getPosition())); - byte[] expected = "abc".getBytes(); - page.put(0, expected, 0); - - instance.flush(); - - byte[] result = new byte[3]; - page = instance.get(position, virtualPageFile.getFilePath(), pageKey -> virtualPageFile.mappedPage(pageKey.getPosition())); - page.get(0, result, 0); - - assertArrayEquals(expected, result); - } - - @Test - public void testGetPageSize() { - setup(false); - assertEquals(512, instance.getPageSize()); - } -} diff --git a/src/test/java/com/upserve/uppend/blobs/VirtualAppendOnlyBlobStoreTest.java b/src/test/java/com/upserve/uppend/blobs/VirtualAppendOnlyBlobStoreTest.java index 42f48764..f3a38e20 100644 --- a/src/test/java/com/upserve/uppend/blobs/VirtualAppendOnlyBlobStoreTest.java +++ b/src/test/java/com/upserve/uppend/blobs/VirtualAppendOnlyBlobStoreTest.java @@ -3,6 +3,7 @@ import com.google.common.primitives.Longs; import com.upserve.uppend.util.SafeDeleting; import org.junit.*; +import org.junit.rules.ExpectedException; import java.io.IOException; import java.nio.file.*; @@ -15,32 +16,110 @@ public class VirtualAppendOnlyBlobStoreTest { private String name = "blobs_test"; - private Path rootPath = Paths.get("build/test/blobStore"); + private Path rootPath = Paths.get("build/test/blobs/blob_store"); private Path blobsPath = rootPath.resolve(name); private VirtualPageFile virtualPageFile; - private ExecutorService executorService; private static int NUMBER_OF_STORES = 13; + @Rule + public ExpectedException thrown = ExpectedException.none(); + @Before public void initialize() throws IOException { SafeDeleting.removeDirectory(rootPath); Files.createDirectories(rootPath); - executorService = new ForkJoinPool(); + } + private void setup(int pageSize) { + setup(pageSize, false); } - public void setup(int pageSize) { - PageCache pageCache = new PageCache(pageSize, 1024, 4096, executorService, null); - virtualPageFile = new VirtualPageFile(blobsPath, NUMBER_OF_STORES, false, pageCache); + private void setup(int pageSize, boolean readOnly) { + virtualPageFile = new VirtualPageFile(blobsPath, NUMBER_OF_STORES, pageSize, 16384, readOnly); } @After public void uninitialize() throws IOException { virtualPageFile.close(); - executorService.shutdown(); + } + + @Test + public void testGetPosition() { + setup(4); + VirtualAppendOnlyBlobStore store = new VirtualAppendOnlyBlobStore(2, virtualPageFile); + + assertEquals(0, store.getPosition()); + store.append("abc".getBytes()); + assertEquals(7, store.getPosition()); + } + + @Test + public void testAppendReadEmpty() { + setup(12); + VirtualAppendOnlyBlobStore store = new VirtualAppendOnlyBlobStore(2, virtualPageFile); + + long pos = store.append(new byte[]{}); + assertArrayEquals(new byte[]{}, store.read(pos)); + } + + @Test + public void testAppendReadEmptyAtPageBoundary() { + setup(4); // the empty value is on a new page + VirtualAppendOnlyBlobStore store = new VirtualAppendOnlyBlobStore(2, virtualPageFile); + + long pos = store.append(new byte[]{}); + assertArrayEquals(new byte[]{}, store.read(pos)); + } + + @Test + public void testAppendReadAbc() { + setup(4); + VirtualAppendOnlyBlobStore store = new VirtualAppendOnlyBlobStore(2, virtualPageFile); + + long pos = store.append("abc".getBytes()); + assertArrayEquals("abc".getBytes(), store.read(pos)); + } + + @Test + public void testReadPastPositionInValidPage() { + setup(25); + VirtualAppendOnlyBlobStore store = new VirtualAppendOnlyBlobStore(2, virtualPageFile); + + long pos = store.append("abc".getBytes()); + assertArrayEquals(new byte[]{}, store.read(pos + 10)); + } + + @Test + public void testReadPastEndOfPage() { + setup(4); + VirtualAppendOnlyBlobStore store = new VirtualAppendOnlyBlobStore(2, virtualPageFile); + + long pos = store.append("abc".getBytes()); + thrown.expect(IllegalStateException.class); + store.read(pos + 10); + } + + @Test + public void testReadBadPos() { + setup(4); + VirtualAppendOnlyBlobStore store = new VirtualAppendOnlyBlobStore(2, virtualPageFile); + + long pos = store.append("abc".getBytes()); + thrown.expect(IllegalStateException.class); + store.read(pos +1); + } + + @Test + public void testReadNegativePos() { + setup(4); + VirtualAppendOnlyBlobStore store = new VirtualAppendOnlyBlobStore(2, virtualPageFile); + + long pos = store.append("abc".getBytes()); + thrown.expect(IllegalArgumentException.class); + store.read(-1); } @Test @@ -110,7 +189,6 @@ public void testClose() throws IOException { public void testConcurrent() { setup(1280); IntStream.range(0, NUMBER_OF_STORES) - .parallel() .forEach(this::concurrentHelper); } @@ -119,7 +197,7 @@ private void concurrentHelper(int virtualBlobStoreNumber) { ConcurrentMap testData = new ConcurrentHashMap<>(); - LongStream.range(0, 10_000) + LongStream.range(0, 200_000) .parallel() .forEach(val -> { byte[] bytes = Longs.toByteArray(val); diff --git a/src/test/java/com/upserve/uppend/blobs/VirtualLongBlobStoreTest.java b/src/test/java/com/upserve/uppend/blobs/VirtualLongBlobStoreTest.java index 071da7fc..7be557b1 100644 --- a/src/test/java/com/upserve/uppend/blobs/VirtualLongBlobStoreTest.java +++ b/src/test/java/com/upserve/uppend/blobs/VirtualLongBlobStoreTest.java @@ -15,7 +15,7 @@ public class VirtualLongBlobStoreTest { private String name = "long_blobs_test"; - private Path rootPath = Paths.get("build/test/blobStore"); + private Path rootPath = Paths.get("build/test/blobs/long_blob_store"); private Path blobsPath = rootPath.resolve(name); private VirtualPageFile virtualPageFile; @@ -32,8 +32,7 @@ public void initialize() throws IOException { } private void setup(int pageSize) { - PageCache pageCache = new PageCache(pageSize, 1024, 4096, executorService, null); - virtualPageFile = new VirtualPageFile(blobsPath, NUMBER_OF_STORES, false, pageCache); + virtualPageFile = new VirtualPageFile(blobsPath, NUMBER_OF_STORES, pageSize,16384,false); } @After @@ -132,7 +131,6 @@ public void testPageAlignment() throws IOException { assertArrayEquals(new byte[]{0, 0, 0, 13, 0, 0, 0, 0, 0, 0, 0, 1, 109, 95, 48, 48, 48, 48, 49, 95, 48, 48, 48, 48, 48, 0, 0, 0, 13, 0, 0, 0, 0, 0, 0, 0, 2, 110, 95, 48}, bytes); page1.get(0, bytes, 0); assertArrayEquals(new byte[]{48, 48, 48, 49, 95, 48, 48, 48, 48, 49, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, bytes); - } @Test @@ -163,19 +161,17 @@ public void testAdjustedPageAlignment() throws IOException { @Test public void testConcurrent() { - setup(1280); + setup(657); IntStream.range(0, NUMBER_OF_STORES) - .parallel() .forEach(this::concurrentHelper); } private void concurrentHelper(int virtualBlobStoreNumber) { - VirtualLongBlobStore blobStore = new VirtualLongBlobStore(virtualBlobStoreNumber, virtualPageFile); ConcurrentMap testData = new ConcurrentHashMap<>(); - LongStream.range(0, 10_000) + LongStream.range(0, 200_000) .parallel() .forEach(val -> { byte[] bytes = Longs.toByteArray(val); diff --git a/src/test/java/com/upserve/uppend/blobs/VirtualMutableBlobStoreTest.java b/src/test/java/com/upserve/uppend/blobs/VirtualMutableBlobStoreTest.java new file mode 100644 index 00000000..6c0bb552 --- /dev/null +++ b/src/test/java/com/upserve/uppend/blobs/VirtualMutableBlobStoreTest.java @@ -0,0 +1,91 @@ +package com.upserve.uppend.blobs; + +import com.upserve.uppend.TestHelper; +import com.upserve.uppend.util.SafeDeleting; +import org.junit.*; +import org.junit.rules.ExpectedException; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.*; +import java.util.concurrent.*; + +import static org.junit.Assert.assertEquals; + +public class VirtualMutableBlobStoreTest { + private String name = "mutable_blobs_test"; + private Path rootPath = Paths.get("build/test/blobs/mutable_blob_store"); + private Path mutableBlobsPath = rootPath.resolve(name); + + private VirtualPageFile virtualPageFile; + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + private ExecutorService executorService; + private static int NUMBER_OF_STORES = 13; + + @Before + public void initialize() throws IOException { + + SafeDeleting.removeDirectory(rootPath); + Files.createDirectories(rootPath); + executorService = new ForkJoinPool(); + } + + private void setup(int pageSize) { + virtualPageFile = new VirtualPageFile(mutableBlobsPath, NUMBER_OF_STORES, pageSize,16384,false); + } + + @After + public void uninitialize() throws IOException { + virtualPageFile.close(); + executorService.shutdown(); + } + + @Test + public void testWriteRead() { + setup(64); + VirtualMutableBlobStore store = new VirtualMutableBlobStore(2, virtualPageFile); + + // Write the buffer to a position thar crosses 3 pages + byte[] bytes = TestHelper.genBytes(123); + store.write(15, bytes); + byte[] result = store.read(15); + + assertEquals(ByteBuffer.wrap(bytes), ByteBuffer.wrap(result)); + } + + @Test + public void testWriteRead_badChecksum() { + setup(64); + VirtualMutableBlobStore store = new VirtualMutableBlobStore(2, virtualPageFile); + + byte[] bytes = TestHelper.genBytes(123); + store.write(15, bytes); + + // overwrite part of the hash + VirtualPageFileIO pageFileIO = new VirtualPageFileIO(2, virtualPageFile); + pageFileIO.writeInt(19, 88); // Visible in warning message as [0, 0, 0, 88] + + thrown.expect(IllegalStateException.class); + store.read(15); + } + + @Test + public void testWriteRead_badBytes() { + setup(64); + VirtualMutableBlobStore store = new VirtualMutableBlobStore(2, virtualPageFile); + + byte[] bytes = TestHelper.genBytes(123); + store.write(15, bytes); + + // overwrite part of the hash + VirtualPageFileIO pageFileIO = new VirtualPageFileIO(2, virtualPageFile); + pageFileIO.writeInt(26, 88); // [... 0, 0, 0, 88... ] visible in warning message + + thrown.expect(IllegalStateException.class); + store.read(15); + } + // TODO Add test for concurrent read write access +} diff --git a/src/test/java/com/upserve/uppend/blobs/VirtualPageFileTest.java b/src/test/java/com/upserve/uppend/blobs/VirtualPageFileTest.java index 19ef7d79..db8e1f2d 100644 --- a/src/test/java/com/upserve/uppend/blobs/VirtualPageFileTest.java +++ b/src/test/java/com/upserve/uppend/blobs/VirtualPageFileTest.java @@ -6,12 +6,12 @@ import java.io.IOException; import java.nio.file.*; +import static junit.framework.TestCase.assertEquals; import static junit.framework.TestCase.assertTrue; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertFalse; public class VirtualPageFileTest { - private String name = "virtual_page_file_test"; private Path rootPath = Paths.get("build/test/blobs/virtual_page_file"); private Path path = rootPath.resolve(name); @@ -31,16 +31,20 @@ public void teardown() throws IOException { @Test public void testReadWritePageAllocation() throws IOException { - - instance = new VirtualPageFile(path, 36, 1024, false); + instance = new VirtualPageFile(path, 36, 1024, 16384, false); byte[] result = new byte[3]; assertFalse(instance.isPageAvailable(0, 0)); assertFalse(instance.isPageAvailable(18, 0)); - Page page5 = instance.getCachedOrCreatePage(0, 5, false); + Page page5 = instance.getOrCreatePage(0, 5); page5.put(16, "abc".getBytes(), 0); + Page page5RO = instance.getExistingPage(0, 5); + + page5RO.get(16,result,0); + assertArrayEquals("abc".getBytes(), result); + assertTrue(instance.isPageAvailable(0, 0)); assertTrue(instance.isPageAvailable(0, 1)); assertTrue(instance.isPageAvailable(0, 2)); @@ -49,12 +53,8 @@ public void testReadWritePageAllocation() throws IOException { assertTrue(instance.isPageAvailable(0, 5)); assertFalse(instance.isPageAvailable(18, 0)); - - Page page4 = instance.getExistingPage(0, 4); - page4.put(12, "def".getBytes(), 0); - instance.close(); - instance = new VirtualPageFile(path, 36, 1024, true); + instance = new VirtualPageFile(path, 36, 1024, 16384,true); assertFalse(instance.isPageAvailable(18, 0)); assertTrue(instance.isPageAvailable(0, 5)); @@ -63,21 +63,73 @@ public void testReadWritePageAllocation() throws IOException { page5.get(16, result, 0); assertArrayEquals("abc".getBytes(), result); - page4 = instance.getExistingPage(0, 4); - page4.get(12, result, 0); - assertArrayEquals("def".getBytes(), result); - instance.close(); - instance = new VirtualPageFile(path, 36, 1024, false); + instance = new VirtualPageFile(path, 36, 1024, 16384,false); - Page page7 = instance.getCachedOrCreatePage(0, 7, false); + Page page7 = instance.getOrCreatePage(0, 7); page7.put(28, "ghi".getBytes(), 0); page7.get(28, result, 0); assertArrayEquals("ghi".getBytes(), result); + } + + @Test + public void testReadOnlyTruncation() throws IOException { + instance = new VirtualPageFile(path, 36, 1024, 16384, false); + Page page = instance.getOrCreatePage(5,0); + page.put(12, "abc".getBytes(), 0); + + page = instance.getExistingPage(5,0); + + assertEquals(313016, instance.getFileSize()); + + instance.close(); + // We can open the file in read only after truncation + VirtualPageFile roInstance = new VirtualPageFile(path, 36, 1024, 16384, true); + assertEquals(313016, roInstance.getFileSize()); + page = roInstance.getExistingPage(5,0); + + byte[] bytes = new byte[3]; + page.get(12, bytes, 0); + assertArrayEquals("abc".getBytes(), bytes); + + // Make a new page - check that file is extended again. + instance = new VirtualPageFile(path, 36, 1024, 16384, false); + assertEquals(313016L, instance.getFileSize()); + + page = instance.getOrCreatePage(6,0); + page.put(6, "def".getBytes(), 0); + page.put(900, "ghi".getBytes(), 0); + + page = roInstance.getExistingPage(6,0); + assertEquals(313016L, roInstance.getFileSize()); + page.get(6, bytes, 0); + assertArrayEquals("def".getBytes(), bytes); + + instance.close(); + + assertEquals(298680L, roInstance.getFileSize()); + + page.get(900, bytes, 0); + assertArrayEquals("ghi".getBytes(), bytes); } + @Test + public void testTableExtension() throws IOException { + instance = new VirtualPageFile(path, 5, 16, 16384, false); + byte[] result = new byte[3]; + instance.getOrCreatePage(0,1000); + assertEquals(1001, instance.getAllocatedPageCount()); + assertTrue("Page should be available",instance.isPageAvailable(0, 1000)); + + instance.close(); + + instance = new VirtualPageFile(path, 5, 16, 16384, false); + + assertEquals(1001, instance.getAllocatedPageCount()); + assertTrue("Page should be available",instance.isPageAvailable(0, 1000)); + } } diff --git a/src/test/java/com/upserve/uppend/cli/BenchmarkTest.java b/src/test/java/com/upserve/uppend/cli/BenchmarkTest.java new file mode 100644 index 00000000..60ea2bef --- /dev/null +++ b/src/test/java/com/upserve/uppend/cli/BenchmarkTest.java @@ -0,0 +1,76 @@ +package com.upserve.uppend.cli; + +import com.upserve.uppend.TestHelper; +import com.upserve.uppend.util.SafeDeleting; +import org.junit.*; +import picocli.CommandLine; + +import java.io.IOException; +import java.nio.file.*; + +import static org.junit.Assert.assertEquals; + +public class BenchmarkTest extends TestHelper.IoStreamHelper { + + CommandBenchmark commandBenchmark; + CommandLine commandLine; + + @Before + public void setUpDir() throws IOException { + commandBenchmark = new CommandBenchmark(); + commandLine = new CommandLine(commandBenchmark).registerConverter(Path.class, (p) -> Paths.get(p)); + SafeDeleting.removeDirectory(Paths.get("build/test/cli")); + } + + @After + public void tearDownDir() throws IOException { + SafeDeleting.removeDirectory(Paths.get("build/test/cli")); + } + + @Test + public void tesUsage() { + commandLine.execute("--help"); + assertStdOutContains("Usage: uppend benchmark [--help] [-b=] [-c=]"); + assertStdOutContains("[-m=] [-s=] "); + } + + @Test + public void testMissingRequiredPath() { + commandLine.execute("-b", "small"); + assertStdErrContains("Missing required parameter: "); + } + + @Test + public void testBadArgument() { + commandLine.execute("--badArg", "pretendPath"); + assertStdErrContains("Unknown option: '--badArg'"); + } + + @Test + public void testBenchmark() { + commandLine.execute("-s", "small", "-b", "small", "build/test/cli/bench"); + assertEquals(1000000L, commandBenchmark.benchmark.writerStats().getCount()); + } + + @Test + public void testBenchmarkWide() { + commandLine.execute("-s", "nano", "-c", "wide", "-b", "small", "build/test/cli/bench"); + assertEquals(20000L, commandBenchmark.benchmark.writerStats().getCount()); + } + + @Test + public void testBenchmarkReadWrite() { + commandLine.execute("-s", "nano", "-m", "readwrite", "-b", "small", "build/test/cli/bench"); + assertEquals(10000L, commandBenchmark.benchmark.writerStats().getCount()); + assertEquals(10000L, commandBenchmark.benchmark.readerStats().getCount()); + } + + @Test + public void testBenchmarkWriteThenRead() { + commandLine.execute("-s", "nano", "-m", "write", "-b", "small", "build/test/cli/bench"); + assertEquals(10000L, commandBenchmark.benchmark.writerStats().getCount()); + + commandLine.execute("-s", "nano", "-m", "read", "-b", "small", "build/test/cli/bench"); + assertEquals(10000L, commandBenchmark.benchmark.readerStats().getCount()); + } +} diff --git a/src/test/java/com/upserve/uppend/cli/CliTest.java b/src/test/java/com/upserve/uppend/cli/CliTest.java index ca51c77d..3b45fcf7 100644 --- a/src/test/java/com/upserve/uppend/cli/CliTest.java +++ b/src/test/java/com/upserve/uppend/cli/CliTest.java @@ -1,99 +1,42 @@ package com.upserve.uppend.cli; -import com.upserve.uppend.util.SafeDeleting; +import com.upserve.uppend.TestHelper; import org.junit.*; -import java.io.*; -import java.nio.file.Paths; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -public class CliTest { - private static final PrintStream origErr = System.err; - private final ByteArrayOutputStream newErrBytes = new ByteArrayOutputStream(); - private final PrintStream newErr = new PrintStream(newErrBytes); - private String err; - - private static final PrintStream origOut = System.out; - private final ByteArrayOutputStream newOutBytes = new ByteArrayOutputStream(); - private final PrintStream newOut = new PrintStream(newOutBytes); - private String out; - - @Before - public void setUp() throws IOException { - SafeDeleting.removeDirectory(Paths.get("build/test/cli/bench")); - System.setErr(newErr); - System.setOut(newOut); - } - - @After - public void tearDown() { - System.setErr(origErr); - System.setOut(origOut); - } - +public class CliTest extends TestHelper.IoStreamHelper { @Test - public void testUsage() throws Exception { + public void testNoArgs() throws Exception { Cli.main(); - syncStreams(); - assertEquals("", out); - assertTrue(err.contains("Usage: uppend")); - } - - @Test - public void testVersion() throws Exception { - Cli.main("version", "--verbose"); - syncStreams(); - assertTrue(out.startsWith("Uppend version")); - assertEquals("", err); + assertStdErrContains("Usage: uppend [--help] [COMMAND]"); } @Test - public void testBenchmark() throws Exception { - Cli.main("benchmark", "-s", "small", "build/test/cli/bench"); - syncStreams(); - assertTrue("expected benchmark output to contain '[benchmark is done]': " + out, out.contains("[benchmark is done]")); - assertEquals("", err); + public void testUsage() throws Exception { + Cli.main("--help"); + assertStdOutContains("Usage: uppend [--help] [COMMAND]"); + assertStdOutContains("benchmark Run store benchmark"); + assertStdOutContains("version Print version information"); + assertStdOutContains("filestore Run file store benchmark"); } @Test - public void testBenchmarkWide() throws Exception { - Cli.main("benchmark", "-s", "nano", "-c", "wide", "build/test/cli/bench"); - syncStreams(); - assertTrue("expected benchmark output to contain '[benchmark is done]': " + out, out.contains("[benchmark is done]")); - assertEquals("", err); + public void testBadCommand() throws Exception { + Cli.main("foobar"); + assertStdErrContains("Unmatched argument at index 0: 'foobar'"); + assertStdErrContains("Did you mean:"); } @Test - public void testBenchmarkReadWrite() throws Exception { - Cli.main("benchmark", "-s", "nano", "-m", "readwrite", "build/test/cli/bench"); - syncStreams(); - assertTrue("expected benchmark output to contain '[benchmark is done]': " + out, out.contains("[benchmark is done]")); - assertEquals("", err); + public void testBadArgument() throws Exception { + Cli.main("--foobar"); + assertStdErrContains("Unknown option: '--foobar'"); + assertStdErrContains("Usage: uppend [--help] [COMMAND]"); } @Test - public void testBenchmarkWriteThenRead() throws Exception { - Cli.main("benchmark", "-s", "nano", "-m", "write", "build/test/cli/bench"); - syncStreams(); - assertTrue("expected benchmark output to contain '[benchmark is done]': " + out, out.contains("[benchmark is done]")); - assertEquals("", err); - Cli.main("benchmark", "-s", "nano", "-m", "read", "build/test/cli/bench"); - syncStreams(); - assertTrue("expected benchmark output to contain '[benchmark is done]': " + out, out.contains("[benchmark is done]")); - assertEquals("", err); - } - - private void syncStreams() { - System.out.flush(); - newOut.flush(); - out = newOutBytes.toString(); - newOutBytes.reset(); - - System.err.flush(); - newErr.flush(); - err = newErrBytes.toString(); - newErrBytes.reset(); + public void testExecutionException() throws Exception { + // Assert that an exception that occurs in method execution gets presented nicely + Cli.main("filestore", "-n", "-1", "build/test/cli"); + assertStdErrContains("java.lang.IllegalArgumentException: virtualFiles must be greater than 0 in file"); } } diff --git a/src/test/java/com/upserve/uppend/cli/CommandVersionTest.java b/src/test/java/com/upserve/uppend/cli/CommandVersionTest.java new file mode 100644 index 00000000..2d5de479 --- /dev/null +++ b/src/test/java/com/upserve/uppend/cli/CommandVersionTest.java @@ -0,0 +1,26 @@ +package com.upserve.uppend.cli; + +import com.upserve.uppend.TestHelper; +import org.junit.*; +import picocli.CommandLine; + +public class CommandVersionTest extends TestHelper.IoStreamHelper { + + @Test + public void testVersion() throws Exception { + new CommandLine(new CommandVersion()).execute(); + assertStdOutContains("test-version"); + } + + @Test + public void testVersionHelp() throws Exception { + new CommandLine(new CommandVersion()).execute("--help"); + assertStdOutContains("Usage: uppend version [-v] [--help]"); + } + + @Test + public void testVersionVerbose() throws Exception { + new CommandLine(new CommandVersion()).execute("-v"); + assertStdOutContains("Uppend version test-version"); + } +} diff --git a/src/test/java/com/upserve/uppend/cli/FileStoreBenchmarkTest.java b/src/test/java/com/upserve/uppend/cli/FileStoreBenchmarkTest.java new file mode 100644 index 00000000..cb8beca0 --- /dev/null +++ b/src/test/java/com/upserve/uppend/cli/FileStoreBenchmarkTest.java @@ -0,0 +1,53 @@ +package com.upserve.uppend.cli; + +import com.upserve.uppend.TestHelper; +import com.upserve.uppend.util.SafeDeleting; +import org.junit.*; +import picocli.CommandLine; + +import java.io.IOException; +import java.nio.file.*; + +import static org.junit.Assert.assertEquals; + +public class FileStoreBenchmarkTest extends TestHelper.IoStreamHelper { + CommandFileStoreBenchmark commandBenchmark; + CommandLine commandLine; + + @Before + public void setUpDir() throws IOException { + commandBenchmark = new CommandFileStoreBenchmark(); + commandLine = new CommandLine(commandBenchmark).registerConverter(Path.class, (p) -> Paths.get(p)); + SafeDeleting.removeDirectory(Paths.get("build/test/cli")); + } + + @After + public void tearDownDir() throws IOException { + SafeDeleting.removeDirectory(Paths.get("build/test/cli")); + } + + @Test + public void tesUsage() { + commandLine.execute("--help"); + assertStdOutContains("Usage: uppend filestore [--help] [-b=] [-n=]"); + assertStdOutContains("[-p=] [-s=] "); + } + + @Test + public void testMissingRequiredPath() { + commandLine.execute("-b", "small"); + assertStdErrContains("Missing required parameter: "); + } + + @Test + public void testBadArgument() { + commandLine.execute("--badArg", "pretendPath"); + assertStdErrContains("Unknown option: '--badArg'"); + } + + @Test + public void testSmall() { + commandLine.execute("-b", "small", "-n", "4", "-s", "small", "build/test/cli"); + assertEquals(1000000, commandBenchmark.getStats().getCount()); + } +} diff --git a/src/test/java/com/upserve/uppend/lookup/LookupDataTest.java b/src/test/java/com/upserve/uppend/lookup/LookupDataTest.java index a27d534f..6e52f401 100644 --- a/src/test/java/com/upserve/uppend/lookup/LookupDataTest.java +++ b/src/test/java/com/upserve/uppend/lookup/LookupDataTest.java @@ -1,6 +1,5 @@ package com.upserve.uppend.lookup; -import com.github.benmanes.caffeine.cache.stats.CacheStats; import com.google.common.primitives.Ints; import com.upserve.uppend.*; import com.upserve.uppend.blobs.*; @@ -12,33 +11,19 @@ import java.nio.file.*; import java.util.*; import java.util.concurrent.*; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.*; import static org.junit.Assert.*; public class LookupDataTest { - - private static final String LOOKUP_KEY = "Lookup Key"; - private static final String LOOKUP_PAGES = "Lookup Pages"; - private static final String LOOKUP_METADATA = "Lookup Metadata"; + private static final int RELOAD_INTERVAL = -1; + private static final int FLUSH_THRESHOLD = -1; private final String name = "lookupdata-test"; private final Path lookupDir = Paths.get("build/test/lookup").resolve(name); private AppendOnlyStoreBuilder defaults = TestHelper - .getDefaultTestBuilder() - .withMaximumLookupKeyCacheWeight(1024 * 1024) - .withMaximumMetaDataCacheWeight(500_000); - - private final PageCache pageCache = defaults.buildLookupPageCache(name); - private final LookupCache lookupCache = defaults.buildLookupCache(name); - - private final PartitionLookupCache partitionLookupCache = PartitionLookupCache.create("partition", lookupCache); - - private AtomicReference lookupPageCacheStats = new AtomicReference<>(pageCache.stats()); - private AtomicReference lookupKeyCacheStats = new AtomicReference<>(lookupCache.keyStats()); - private AtomicReference lookupMetadataCacheStats = new AtomicReference<>(lookupCache.metadataStats()); + .getDefaultAppendStoreTestBuilder(); private VirtualPageFile metadataPageFile; private VirtualMutableBlobStore mutableBlobStore; @@ -58,19 +43,16 @@ public void initialize() throws Exception { setup(false); } - public void setup(boolean readOnly) { - metadataPageFile = new VirtualPageFile(lookupDir.resolve("metadata"), NUMBER_OF_STORES, 1024, readOnly); + metadataPageFile = new VirtualPageFile(lookupDir.resolve("metadata"), NUMBER_OF_STORES, 1024, 16384, readOnly); mutableBlobStore = new VirtualMutableBlobStore(1, metadataPageFile); - keyDataPageFile = new VirtualPageFile(lookupDir.resolve("keydata"), NUMBER_OF_STORES, readOnly, pageCache); + keyDataPageFile = new VirtualPageFile(lookupDir.resolve("keydata"), NUMBER_OF_STORES, defaults.getLookupPageSize(), defaults.getTargetBufferSize(), readOnly); keyBlobStore = new VirtualLongBlobStore(1, keyDataPageFile); } @After public void tearDown() throws IOException { - lookupCache.flush(); - pageCache.flush(); keyDataPageFile.close(); metadataPageFile.close(); } @@ -79,7 +61,7 @@ public void tearDown() throws IOException { public void testOpenEmptyReadOnly() throws IOException { tearDown(); // Close the page files setup(true); - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, true); + LookupData data = LookupData.lookupReader(keyBlobStore, mutableBlobStore, RELOAD_INTERVAL); final LookupKey key = new LookupKey("mykey"); assertNull(data.getValue(key)); @@ -89,7 +71,7 @@ public void testOpenEmptyReadOnly() throws IOException { @Test public void testOpenGetAndPut() { - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); + LookupData data = LookupData.lookupWriter(keyBlobStore, mutableBlobStore, FLUSH_THRESHOLD); final LookupKey key = new LookupKey("mykey"); assertNull(data.getValue(key)); data.put(key, 80); @@ -98,8 +80,9 @@ public void testOpenGetAndPut() { @Test public void testPutIfNotExists() { - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); + LookupData data = LookupData.lookupWriter(keyBlobStore, mutableBlobStore, FLUSH_THRESHOLD); final LookupKey key = new LookupKey("mykey"); + assertNull("The key should not exist yet", data.getValue(key)); data.putIfNotExists(key, 1); assertEquals(Long.valueOf(1), data.getValue(key)); data.putIfNotExists(key, 2); @@ -108,8 +91,9 @@ public void testPutIfNotExists() { @Test public void testPutIfNotExistsFunction() { - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); + LookupData data = LookupData.lookupWriter(keyBlobStore, mutableBlobStore, FLUSH_THRESHOLD); final LookupKey key = new LookupKey("mykey"); + assertNull("The key should not exist yet", data.getValue(key)); data.putIfNotExists(key, () -> 1); assertEquals(Long.valueOf(1), data.getValue(key)); data.putIfNotExists(key, () -> 2); @@ -117,378 +101,111 @@ public void testPutIfNotExistsFunction() { } @Test - public void testFlushAndClose() throws IOException { - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); + public void testPut() { + LookupData data = LookupData.lookupWriter(keyBlobStore, mutableBlobStore, FLUSH_THRESHOLD); final LookupKey key = new LookupKey("mykey"); - data.put(key, 80); - data.flush(); - - lookupCache.flush(); - pageCache.flush(); - - Long result = data.getValue(key); - assertEquals(Long.valueOf(80), result); - - tearDown(); - setup(true); - - data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, true); - result = data.getValue(key); - assertEquals(Long.valueOf(80), result); - } - - @Test - public void testCachePutSupplierIfNotExistFlush() throws IOException { - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); - - final LookupKey key = new LookupKey("mykey"); - assertEquals(16, data.putIfNotExists(key, () -> 16L)); - - assertEquals(Long.valueOf(16), data.writeCache.get(key)); - - assertLookupKeyCache(0, 1, 0, 1); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 1, 1, 0); - - // ignores new value - nothing changes - assertEquals(16, data.putIfNotExists(key, () -> 17L)); - - assertEquals(Long.valueOf(16), data.writeCache.get(key)); - - assertLookupKeyCache(0, 0, 0, 0); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 0, 0, 0); - - // Flush the write Cache and put the key in the read cache - data.flush(); - - assertNull(data.writeCache.get(key)); - - assertLookupKeyCache(0, 0, 0, 0); - assertLookupPagesCache(0, 1, 0, 0); - assertLookupMetadataCache(1, 0, 0, 0); - - // call put if not exist again with the data on disk and in the read caches - assertEquals(16, data.putIfNotExists(key, () -> 17L)); - - assertLookupKeyCache(1, 0, 0, 0); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 0, 0, 0); - - lookupCache.flush(); - - // call put if not exist again with the data on disk but not in the read cache - assertEquals(16, data.putIfNotExists(key, () -> 17L)); - - assertLookupKeyCache(0, 1, 1, 0); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 1, 1, 0); - - tearDown(); - setup(false); - - data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); - - assertEquals(16, data.putIfNotExists(key, () -> 18L)); - - assertLookupKeyCache(0, 1, 1, 0); - assertLookupPagesCache(0, 1, 1, 0); - assertLookupMetadataCache(0, 1, 1, 0); + assertNull("The key should not exist yet", data.getValue(key)); + data.put(key, 1); + assertEquals(Long.valueOf(1), data.getValue(key)); + data.put(key, 2); + assertEquals(Long.valueOf(2), data.getValue(key)); } @Test - public void testCachePutValIfNotExistFlush() throws IOException { - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); - + public void testIncrement() { + LookupData data = LookupData.lookupWriter(keyBlobStore, mutableBlobStore, FLUSH_THRESHOLD); final LookupKey key = new LookupKey("mykey"); - assertEquals(80, data.putIfNotExists(key, 80)); - - assertEquals(Long.valueOf(80), data.writeCache.get(key)); - - assertLookupKeyCache(0, 1, 0, 1); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 1, 1, 0); - - // ignores new value - nothing changes - assertEquals(80, data.putIfNotExists(key, 86)); - - assertEquals(Long.valueOf(80), data.writeCache.get(key)); - - assertLookupKeyCache(0, 0, 0, 0); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 0, 0, 0); - - // Flush the write Cache and put the key in the read cache - data.flush(); - - assertNull(data.writeCache.get(key)); - - assertLookupKeyCache(0, 0, 0, 0); - assertLookupPagesCache(0, 1, 0, 0); - assertLookupMetadataCache(1, 0, 0, 0); - - // call put if not exist again with the data on disk and in the read caches - assertEquals(80, data.putIfNotExists(key, 86)); - - assertLookupKeyCache(1, 0, 0, 0); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 0, 0, 0); - - lookupCache.flush(); - - // call put if not exist again with the data on disk but not in the read cache - assertEquals(80, data.putIfNotExists(key, 86)); - - assertLookupKeyCache(0, 1, 1, 0); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 1, 1, 0); - - tearDown(); - setup(false); - - data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); - - assertEquals(80, data.putIfNotExists(key, 87)); - - assertLookupKeyCache(0, 1, 1, 0); - assertLookupPagesCache(0, 1, 1, 0); - assertLookupMetadataCache(0, 1, 1, 0); + assertNull("The key should not exist yet", data.getValue(key)); + data.increment(key, 10); + assertEquals(Long.valueOf(10), data.getValue(key)); + data.increment(key, 2); + assertEquals(Long.valueOf(12), data.getValue(key)); } @Test - public void testCachePutFlush() throws IOException, InterruptedException { - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); - - final LookupKey key = new LookupKey("mykey"); - assertNull(data.put(key, 80)); - - assertEquals(Long.valueOf(80), data.writeCache.get(key)); - - assertLookupKeyCache(0, 1, 0, 1); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 1, 1, 0); - - // Updating the value while still in the write cache changes nothing else - assertEquals(Long.valueOf(80), data.put(key, 81)); - - assertEquals(Long.valueOf(81), data.writeCache.get(key)); // write cache is updated... - - assertLookupKeyCache(0, 0, 0, 0); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 0, 0, 0); - - - // Flush the key to disk and more the key/value to the read cache - data.flush(); - - assertNull(data.writeCache.get(key)); // the key has been written and moved from the write cache to the read cache - - assertLookupKeyCache(0, 0, 0, 0); - assertLookupPagesCache(0, 1, 0, 0); - assertLookupMetadataCache(1, 0, 0, 0); - - // put a new value and see which cache entries change - assertEquals(Long.valueOf(81), data.put(key, 82)); - - assertNull(data.writeCache.get(key)); // Write cache is only for new keys. - - assertLookupKeyCache(1, 0, 0, 0); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 0, 0, 0); - - lookupCache.flush(); - - assertEquals(Long.valueOf(82), data.put(key, 83)); - - assertNull(data.writeCache.get(key)); // Write cache is only for new keys - - assertLookupKeyCache(0, 1, 1, 0); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 1, 1, 0); - - tearDown(); - setup(false); - - data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); - - assertEquals(Long.valueOf(83), data.put(key, 84)); - - assertLookupKeyCache(0, 1, 1, 0); - assertLookupPagesCache(0, 1, 1, 0); - assertLookupMetadataCache(0, 1, 1, 0); - } - - @Test - public void testCacheIncrementFlush() throws IOException { - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); - + public void testFlushAndClose() throws IOException { + LookupData data = LookupData.lookupWriter(keyBlobStore, mutableBlobStore, FLUSH_THRESHOLD); final LookupKey key = new LookupKey("mykey"); - assertEquals(12, data.increment(key, 12)); - - assertEquals(Long.valueOf(12), data.writeCache.get(key)); - - assertLookupKeyCache(0, 1, 0, 1); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 1, 1, 0); - - // Updating the value while still in the write cache changes nothing else - assertEquals(24, data.increment(key, 12)); - - assertEquals(Long.valueOf(24), data.writeCache.get(key)); // write cache is updated... - - assertLookupKeyCache(0, 0, 0, 0); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 0, 0, 0); - - - // Flush the key to disk and more the key/value to the read cache + data.put(key, 80); data.flush(); - assertNull(data.writeCache.get(key)); // the key has been written and moved from the write cache to the read cache - - assertLookupKeyCache(0, 0, 0, 0); - assertLookupPagesCache(0, 1, 0, 0); // Keys blob store and LongLongStore - assertLookupMetadataCache(1, 0, 0, 0); // Metadata is loaded during flush - - // put a new value and see which cache entries change - assertEquals(36, data.increment(key, 12)); - - assertNull(data.writeCache.get(key)); // Write cache is only for new keys. - - assertLookupKeyCache(1, 0, 0, 0); - assertLookupPagesCache(0, 0, 0, 0); // loads the page to write to - assertLookupMetadataCache(0, 0, 0, 0); - - lookupCache.flush(); - - assertEquals(48, data.increment(key, 12)); - - assertNull(data.writeCache.get(key)); // Write cache is only for new keys - - assertLookupKeyCache(0, 1, 1, 0); - assertLookupPagesCache(0, 0, 0, 0); // load the page, then update - assertLookupMetadataCache(0, 1, 1, 0); + Long result = data.getValue(key); + assertEquals(Long.valueOf(80), result); tearDown(); - setup(false); - - data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); - - assertEquals(60, data.increment(key, 12)); - - assertLookupKeyCache(0, 1, 1, 0); - assertLookupPagesCache(0, 1, 1, 0); - assertLookupMetadataCache(0, 1, 1, 0); - } - - // Test helpers - private void assertLookupKeyCache(long hitCount, long missCount, long loadSuccessCount, long loadFailureCount) { - CacheStats current = lookupCache.keyStats(); - assertCache(LOOKUP_KEY, current.minus(lookupKeyCacheStats.getAndSet(current)), hitCount, missCount, loadSuccessCount, loadFailureCount); - } - - private void assertLookupPagesCache(long hitCount, long missCount, long loadSuccessCount, long loadFailureCount) { - CacheStats current = pageCache.stats(); - assertCache(LOOKUP_PAGES, current.minus(lookupPageCacheStats.getAndSet(current)), hitCount, missCount, loadSuccessCount, loadFailureCount); - } - - private void assertLookupMetadataCache(long hitCount, long missCount, long loadSuccessCount, long loadFailureCount) { - CacheStats current = lookupCache.metadataStats(); - assertCache(LOOKUP_METADATA, current.minus(lookupMetadataCacheStats.getAndSet(current)), hitCount, missCount, loadSuccessCount, loadFailureCount); - } + setup(true); - private void assertCache(String name, CacheStats stats, long hitCount, long missCount, long loadSuccessCount, long loadFailureCount) { - if (hitCount > 0) assertEquals(name + " Cache Hit Count", hitCount, stats.hitCount()); - if (missCount > 0) assertEquals(name + " Cache Miss Count", missCount, stats.missCount()); - if (loadSuccessCount > 0) - assertEquals(name + " Cache Load Success Count", loadSuccessCount, stats.loadSuccessCount()); - if (loadFailureCount > 0) - assertEquals(name + " Cache Load Failure Count", loadFailureCount, stats.loadFailureCount()); + data = LookupData.lookupReader(keyBlobStore, mutableBlobStore, RELOAD_INTERVAL); + result = data.getValue(key); + assertEquals(Long.valueOf(80), result); } @Test public void testWriteCacheUnderLoad() throws IOException { - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); + LookupData data = LookupData.lookupWriter(keyBlobStore, mutableBlobStore, FLUSH_THRESHOLD); LongStream.range(0, 100_000) .forEach(val -> { data.putIfNotExists(new LookupKey(String.valueOf(val)), val); }); - assertEquals(100_000, data.writeCache.size()); - assertLookupKeyCache(0, 100_000, 0, 100_000); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(99_999, 1, 1, 0); - data.flush(); assertEquals(0, data.writeCache.size()); - assertLookupKeyCache(0, 0, 0, 0); - assertLookupPagesCache(0, 104, 0, 0); - assertLookupMetadataCache(1, 0, 0, 0); - LongStream.range(0, 100_000) .forEach(val -> { data.putIfNotExists(new LookupKey(String.valueOf(val)), val); }); - assertLookupKeyCache(100_000, 0, 0, 0); - assertLookupPagesCache(0, 0, 0, 0); - assertLookupMetadataCache(0, 0, 0, 0); - - lookupCache.flush(); - pageCache.flush(); - - LongStream.range(0, 100_000) + LongStream.range(0, 100_010) .forEach(val -> { data.putIfNotExists(new LookupKey(String.valueOf(val)), val); }); - - assertLookupKeyCache(0, 100_000, 100_000, 0); - assertLookupPagesCache(233307, 104, 104, 0); - assertLookupMetadataCache(99_999, 1, 1, 0); + assertEquals(10, data.writeCache.size()); } @Test public void testScan() throws IOException { - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); + LookupData data = LookupData.lookupWriter(keyBlobStore, mutableBlobStore, FLUSH_THRESHOLD); LookupKey firstKey = new LookupKey("mykey1"); LookupKey secondKey = new LookupKey("mykey2"); data.put(firstKey, 1); data.put(secondKey, 2); - // Assert that scan works properly during each each phase of the flush operation - LookupMetadata currentMetadata = partitionLookupCache.getMetadata(data); - assertEquals(0, data.flushCache.size()); assertEquals(2, data.writeCache.size()); - data.flushWriteCache(currentMetadata); scanTestHelper(data, new LookupKey[]{firstKey, secondKey}, new Long[]{1L, 2L}); + data.flushWriteCache(data.getMetadata()); + assertEquals(2, data.flushCache.size()); assertEquals(0, data.writeCache.size()); - data.generateMetaData(currentMetadata); scanTestHelper(data, new LookupKey[]{firstKey, secondKey}, new Long[]{1L, 2L}); + data.generateMetaData(data.getMetadata()); + assertEquals(2, data.flushCache.size()); assertEquals(0, data.writeCache.size()); - data.flushCacheToReadCache(); scanTestHelper(data, new LookupKey[]{firstKey, secondKey}, new Long[]{1L, 2L}); + data.flushCache.clear(); + assertEquals(0, data.flushCache.size()); assertEquals(0, data.writeCache.size()); + scanTestHelper(data, new LookupKey[]{firstKey, secondKey}, new Long[]{1L, 2L}); + tearDown(); setup(true); - data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); + data = LookupData.lookupWriter(keyBlobStore, mutableBlobStore, FLUSH_THRESHOLD); scanTestHelper(data, new LookupKey[]{firstKey, secondKey}, new Long[]{1L, 2L}); } @@ -509,10 +226,9 @@ private void scanTestHelper(LookupData data, LookupKey[] expectedKeys, Long[] ex assertArrayEquals(expectedKeys, entries.keySet().stream().sorted().toArray(LookupKey[]::new)); } - @Test public void testScanNonExistant() { - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); + LookupData data = LookupData.lookupWriter(keyBlobStore, mutableBlobStore, FLUSH_THRESHOLD); data.scan((k, v) -> { throw new IllegalStateException("should not have called this"); }); @@ -528,7 +244,7 @@ public void testScanNonExistant() { @Test public void testLoadReadOnlyMetadata() { - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, true); + LookupData data = LookupData.lookupReader(keyBlobStore, mutableBlobStore, RELOAD_INTERVAL); mutableBlobStore.write(0, Ints.toByteArray(50)); mutableBlobStore.write(4, Ints.toByteArray(284482732)); // Check checksum @@ -541,7 +257,7 @@ public void testLoadReadOnlyMetadata() { @Test public void testLoadReadRepairMetadata() { - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, partitionLookupCache, false); + LookupData data = LookupData.lookupWriter(keyBlobStore, mutableBlobStore, FLUSH_THRESHOLD); Random random = new Random(); LongStream.range(0, 100_000) @@ -554,17 +270,13 @@ public void testLoadReadRepairMetadata() { data.flush(); - LookupMetadata expected = lookupCache.getMetadata(data); + LookupMetadata expected = data.getMetadata(); mutableBlobStore.write(0, Ints.toByteArray(50)); - mutableBlobStore.write(4, Ints.toByteArray(284482732)); // Check checksum - - lookupCache.flush(); + mutableBlobStore.write(4, Ints.toByteArray(284482732)); // Invalid Check checksum // Do read repair! - assertNull(data.getValue(new LookupKey("foobarChew - MeToo"))); - - LookupMetadata result = lookupCache.getMetadata(data); + LookupMetadata result = data.loadMetadata(); // It is a new object! assertNotEquals(expected, result); @@ -577,11 +289,7 @@ public void testLoadReadRepairMetadata() { @Test public void testFlushWithAppendLoad() throws ExecutionException, InterruptedException { - - // Force the metadata to be reloaded every time it is needed - LookupCache noCache = defaults.withMaximumMetaDataCacheWeight(0).withMaximumLookupKeyCacheWeight(0).buildLookupCache(name); - - LookupData data = new LookupData(keyBlobStore, mutableBlobStore, PartitionLookupCache.create("partition", noCache), 100, false); + LookupData data = LookupData.lookupWriter(keyBlobStore, mutableBlobStore, 100); int n = 500; @@ -591,7 +299,6 @@ public void testFlushWithAppendLoad() throws ExecutionException, InterruptedExce } }); - Random random = new Random(); Thread writer = new Thread(() -> { for (int j = 0; j < n; j++) { diff --git a/src/test/java/com/upserve/uppend/lookup/LookupMetadataTest.java b/src/test/java/com/upserve/uppend/lookup/LookupMetadataTest.java index 4b74ac98..4c3e1902 100644 --- a/src/test/java/com/upserve/uppend/lookup/LookupMetadataTest.java +++ b/src/test/java/com/upserve/uppend/lookup/LookupMetadataTest.java @@ -21,7 +21,6 @@ @RunWith(MockitoJUnitRunner.class) public class LookupMetadataTest { - private final String name = "lookupMetadata-test"; private final Path rootPath = Paths.get("build/test/lookup/lookupMetadata"); private final Path path = rootPath.resolve(name); @@ -51,7 +50,7 @@ public void before() throws IOException { } public void setup(boolean readOnly) { - virtualPageFile = new VirtualPageFile(path, NUMBER_OF_STORES, 1024, readOnly); + virtualPageFile = new VirtualPageFile(path, NUMBER_OF_STORES, 1024, 16384, readOnly); metadataBlobs = new VirtualMutableBlobStore(1, virtualPageFile); } @@ -85,7 +84,6 @@ public void testInvalidContent() throws Exception { LookupMetadata.open(metadataBlobs, 0); } - @Test public void testCorrectReadWrite() throws Exception { buildSimpleTestData(metadataBlobs); @@ -121,7 +119,6 @@ public void testOpenReadOnly() throws Exception { assertEquals(2, initialMetadata.getMetadataGeneration()); } - @Test public void testEmptyLookup() { LookupMetadata initialMetadata = new LookupMetadata(null, null, new int[0], 1); @@ -135,7 +132,6 @@ public void testEmptyLookup() { assertEquals(-1, searchKey.getInsertAfterSortIndex()); assertEquals(1, searchKey.getMetaDataGeneration()); assertEquals(-1, searchKey.getPosition()); - } @Test @@ -152,7 +148,6 @@ public void testOneKeyLookupAbove() { assertEquals(-1, searchKey.getInsertAfterSortIndex()); assertEquals(1, searchKey.getMetaDataGeneration()); assertEquals(-1, searchKey.getPosition()); - } @Test @@ -169,7 +164,6 @@ public void testOneKeyLookupBelow() { assertEquals(0, searchKey.getInsertAfterSortIndex()); assertEquals(1, searchKey.getMetaDataGeneration()); assertEquals(-1, searchKey.getPosition()); - } @Test @@ -206,7 +200,6 @@ public void testTwoKeyLookupBelowLower() { assertEquals(-1, searchKey.getInsertAfterSortIndex()); assertEquals(1, searchKey.getMetaDataGeneration()); assertEquals(-1, searchKey.getPosition()); - } @Test @@ -389,7 +382,6 @@ public void testManyKeysAboveLastMidpoint() { LookupKey yKey = new LookupKey("y"); LookupMetadata metadata = new LookupMetadata(bKey, yKey, new int[]{12, 7, 8, 1, 11, 6, 3, 5, 10, 2, 0, 4, 9}, 1); - when(mockLongBlobStore.readBlob(3L)).thenReturn("m".getBytes()); // First midpoint is the 6th sort value => 3 when(mockLongBlobStore.readBlob(2L)).thenReturn("u".getBytes()); // Second midpoint is the 9th sort value => 2 when(mockLongBlobStore.readBlob(5L)).thenReturn("o".getBytes()); // Second midpoint is the 7th sort value => 5 @@ -429,22 +421,15 @@ public void testManyKeysAboveLastMidpoint() { verifyNoMoreInteractions(mockLongBlobStore); } - @Test public void testMetadataLookup() { - AppendOnlyStoreBuilder defaults = TestHelper.getDefaultTestBuilder() - .withLookupPageSize(32 * 1024) - .withMaximumLookupKeyCacheWeight(1024 * 1024); - - PageCache pageCache = defaults.buildLookupPageCache(name); - LookupCache lookupCache = defaults.buildLookupCache(name); - - PartitionLookupCache partitionLookupCache = PartitionLookupCache.create("partition", lookupCache); + AppendOnlyStoreBuilder defaults = TestHelper.getDefaultAppendStoreTestBuilder() + .withLookupPageSize(32 * 1024); - VirtualPageFile keysData = new VirtualPageFile(keysPath, NUMBER_OF_STORES, false, pageCache); + VirtualPageFile keysData = new VirtualPageFile(keysPath, NUMBER_OF_STORES, defaults.getLookupPageSize(), defaults.getTargetBufferSize(), false); VirtualLongBlobStore keyStore = new VirtualLongBlobStore(5, keysData); - LookupData lookupData = new LookupData(keyStore, metadataBlobs, partitionLookupCache, false); + LookupData lookupData = LookupData.lookupWriter(keyStore, metadataBlobs, -1); List keys = Ints.asList(IntStream.range(0, 4000).map(i -> i * 2).toArray()); Collections.shuffle(keys, new Random(1234)); keys.forEach(k -> lookupData.put(new LookupKey(String.valueOf(k)), 1000 + k)); diff --git a/src/test/java/com/upserve/uppend/metrics/CounterStoreWithMetricsTest.java b/src/test/java/com/upserve/uppend/metrics/CounterStoreWithMetricsTest.java index be386a87..1db4b902 100644 --- a/src/test/java/com/upserve/uppend/metrics/CounterStoreWithMetricsTest.java +++ b/src/test/java/com/upserve/uppend/metrics/CounterStoreWithMetricsTest.java @@ -1,9 +1,7 @@ package com.upserve.uppend.metrics; import com.codahale.metrics.MetricRegistry; -import com.github.benmanes.caffeine.cache.stats.CacheStats; import com.upserve.uppend.*; -import com.upserve.uppend.lookup.FlushStats; import org.junit.*; import org.junit.runner.RunWith; import org.mockito.Mock; @@ -46,7 +44,6 @@ public void testGetName() { assertEquals("testStore", instance.getName()); } - @Test public void testIncrement() { assertEquals(0, metrics.timer(MetricRegistry.name("MetricsRoot", UPPEND_COUNTER_STORE, store.getName(), INCREMENT_TIMER_METRIC_NAME)).getCount()); @@ -158,34 +155,6 @@ public void testDeregister() { verify(store).deregister(); } - @Test - public void testGetFlushStats() { - FlushStats v = new FlushStats(0, 0); - when(store.getFlushStats()).thenReturn(v); - assertEquals(v, instance.getFlushStats()); - } - - @Test - public void testGetKeyPageCacheStats() { - CacheStats v = new CacheStats(0, 0, 0, 0, 0, 0, 0); - when(store.getKeyPageCacheStats()).thenReturn(v); - assertEquals(v, instance.getKeyPageCacheStats()); - } - - @Test - public void testGetLookupKeyCacheStats() { - CacheStats v = new CacheStats(0, 0, 0, 0, 0, 0, 0); - when(store.getLookupKeyCacheStats()).thenReturn(v); - assertEquals(v, instance.getLookupKeyCacheStats()); - } - - @Test - public void testGetMetadataCacheStats() { - CacheStats v = new CacheStats(0, 0, 0, 0, 0, 0, 0); - when(store.getMetadataCacheStats()).thenReturn(v); - assertEquals(v, instance.getMetadataCacheStats()); - } - @Test public void testKeyCount() { when(store.keyCount()).thenReturn(5L); diff --git a/src/test/java/com/upserve/uppend/metrics/MetricsStatsCounterTest.java b/src/test/java/com/upserve/uppend/metrics/MetricsStatsCounterTest.java deleted file mode 100644 index d6fa6c74..00000000 --- a/src/test/java/com/upserve/uppend/metrics/MetricsStatsCounterTest.java +++ /dev/null @@ -1,51 +0,0 @@ -package com.upserve.uppend.metrics; - -/* - * Copyright 2016 Ben Manes. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import com.codahale.metrics.MetricRegistry; -import com.github.benmanes.caffeine.cache.*; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -/** - * An example of exporting stats to Dropwizard Metrics (http://metrics.dropwizard.io). - * - * @author ben.manes@gmail.com (Ben Manes) - */ -public final class MetricsStatsCounterTest { - - @Test - public void metrics() { - // Use a registry that is exported using a Reporter (via console, JMX, Graphite, etc) - MetricRegistry registry = new MetricRegistry(); - - // Create the cache with a dedicated, uniquely named stats counter - LoadingCache cache = Caffeine.newBuilder() - .recordStats(() -> new MetricsStatsCounter(registry, "example")) - .build(key -> key); - - // Perform application work - for (int i = 0; i < 4; i++) { - cache.get(1); - } - - // Statistics can be queried and reported on - assertEquals(cache.stats().hitCount(), 3L); - assertEquals(registry.meter("example.hits").getCount(), 3L); - } -} \ No newline at end of file diff --git a/src/test/java/com/upserve/uppend/performance/ArrayLongStreamTest.java b/src/test/java/com/upserve/uppend/performance/ArrayLongStreamTest.java new file mode 100644 index 00000000..7f756d31 --- /dev/null +++ b/src/test/java/com/upserve/uppend/performance/ArrayLongStreamTest.java @@ -0,0 +1,64 @@ +package com.upserve.uppend.performance; + +import org.junit.*; +import java.util.*; +import java.util.concurrent.ThreadLocalRandom; +import java.util.function.*; +import java.util.stream.*; + +import static com.upserve.uppend.performance.StreamTimerMethods.*; + +public class ArrayLongStreamTest { + + private static final int values = 1_000_000; + private final long[] longs = new long[values]; + private final int repeats = 5; + private final Supplier parallelStream = () -> Arrays.stream(longs).parallel(); + private final Supplier sequentialStream = () -> Arrays.stream(longs).sequential(); + + @Before + public void loadStore() { + Arrays.setAll(longs, (v) -> ThreadLocalRandom.current().nextLong(0, 512)); + long val = Arrays.stream(longs, 0, 100).parallel().sum(); + } + + @Test + public void sumTest() { + for (int i=0; i Arrays.stream(longs)).parallel())); + sequentialTime(flatMapSum(LongStream.of(1,2,3,4,5,6,7,8,9,10).flatMap(val -> Arrays.stream(longs)).sequential())); + } + } +} diff --git a/src/test/java/com/upserve/uppend/performance/MultiKeyTest.java b/src/test/java/com/upserve/uppend/performance/MultiKeyTest.java new file mode 100644 index 00000000..4746fa99 --- /dev/null +++ b/src/test/java/com/upserve/uppend/performance/MultiKeyTest.java @@ -0,0 +1,88 @@ +package com.upserve.uppend.performance; + +import com.upserve.uppend.*; +import com.upserve.uppend.util.SafeDeleting; +import org.junit.*; + +import java.io.IOException; +import java.nio.file.*; +import java.util.*; +import java.util.function.Supplier; +import java.util.stream.*; + +import static com.upserve.uppend.performance.StreamTimerMethods.*; + +public class MultiKeyTest { + + private static final int numPartitions = 12; + private static final int hashCount = 32; + private static final int values = 100; + private static final int blobsPerBlock = 64; + private static final int keyCount = 1_000; + + private final Path path = Paths.get("build/test/tmp/performance/multi_key"); + private AppendOnlyStore appendOnlyStore; + private final String[] keys = new String[keyCount]; // Just use a single key - it will be hashed appropriately for partition and hashsize + private final int repeats = 5; + + private final Supplier parallelStream = () -> Arrays.stream(keys).flatMapToLong(key -> appendOnlyStore.read(key, key).mapToLong(bytes -> bytes.length).parallel()).parallel(); + private final Supplier sequentialStream = () -> Arrays.stream(keys).flatMapToLong(key -> appendOnlyStore.read(key, key).mapToLong(bytes -> bytes.length).sequential()).sequential(); + + @Before + public void loadStore() throws IOException { + SafeDeleting.removeTempPath(path); + + appendOnlyStore = new AppendOnlyStoreBuilder() + .withPartitionCount(numPartitions) + .withLongLookupHashCount(hashCount) + .withBlobsPerBlock(blobsPerBlock) + .withDir(path).build(); + + for (int value=0; value appendOnlyStore.append(key, key, bytes)); + } + } + + @After + public void cleanup() throws IOException { + SafeDeleting.removeTempPath(path); + } + + @Test + public void sumTest() { + for (int i=0; i parallelStream = () -> appendOnlyStore.read(partition, key).mapToLong(bytes -> bytes.length).parallel(); + private final Supplier sequentialStream = () -> appendOnlyStore.read(partition, key).mapToLong(bytes -> bytes.length).sequential(); + + @Before + public void loadStore() throws IOException { + SafeDeleting.removeTempPath(path); + + appendOnlyStore = new AppendOnlyStoreBuilder() + .withPartitionCount(partitionCount) + .withLongLookupHashCount(hashCount) + .withBlobsPerBlock(blobsPerBlock) + .withDir(path).build(); + + new Random() + .ints(values, 0, 512) + .parallel() + .mapToObj(TestHelper::genBytes) + .forEach(bytes -> appendOnlyStore.append(partition, key, bytes)); + } + + @After + public void cleanup() throws IOException { + SafeDeleting.removeTempPath(path); + } + + @Test + public void sumTest() { + for (int i=0; i appendOnlyStore.read(partition, key)).mapToLong(bytes -> bytes.length).parallel() + )); + sequentialTime(flatMapSum( + Stream.of(1,2,3,4,5).flatMap(val -> appendOnlyStore.read(partition, key)).mapToLong(bytes -> bytes.length).sequential() + )); + } + } +} diff --git a/src/test/java/com/upserve/uppend/performance/StreamTimerMethods.java b/src/test/java/com/upserve/uppend/performance/StreamTimerMethods.java new file mode 100644 index 00000000..62d14861 --- /dev/null +++ b/src/test/java/com/upserve/uppend/performance/StreamTimerMethods.java @@ -0,0 +1,106 @@ +package com.upserve.uppend.performance; + +import org.slf4j.Logger; + +import java.lang.invoke.MethodHandles; +import java.util.*; +import java.util.concurrent.atomic.LongAdder; +import java.util.function.*; +import java.util.stream.*; + +import static junit.framework.TestCase.assertTrue; + +public class StreamTimerMethods { + private static final Logger log = org.slf4j.LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private static final int repeats = 5; + + public static void parallelTime(double time) { + log.info("Parallel execution time : {}ms", String.format("%8.2f", time)); + } + + public static void sequentialTime(double time) { + log.info("Sequential execution time : {}ms", String.format("%8.2f", time)); + } + + public static double sum(Supplier supplier) { + LongStream[] longStreams = new LongStream[repeats]; + long[] sums = new long[repeats]; + + for (int i=0; i 1_000_000L); + } + + return (toc - tic)/1_000_000.0D; + } + + public static double groupByCounting(Supplier supplier, boolean concurrent) { + LongStream[] longStreams = new LongStream[repeats]; + List> groups = new ArrayList<>(repeats); + + for (int i=0; i group: groups){ + assertTrue("Should be large", group.values().stream().mapToLong(Long::longValue).sum() >= 100_000L); + } + return (toc - tic)/1_000_000.0D; + } + + public static double forEachAdder(Supplier supplier) { + LongStream[] longStreams = new LongStream[repeats]; + long[] sums = new long[repeats]; + LongAdder[] longAdders = new LongAdder[repeats]; + + for (int i=0; i 1_000_000L); + } + return (toc - tic)/1_000_000.0D; + } + + public static double flatMapSum(LongStream stream){ + long tic = System.nanoTime(); + long sum = stream.sum(); + long toc = System.nanoTime(); + + assertTrue("Should be large", sum > 1_000_000L); + + return (toc - tic)/1_000_000.0D; + } +} diff --git a/src/test/resources/log4j2.xml b/src/test/resources/log4j2.xml index 16be9dec..c4be2b0d 100644 --- a/src/test/resources/log4j2.xml +++ b/src/test/resources/log4j2.xml @@ -1,12 +1,12 @@ - + - +