This is an automated email from the ASF dual-hosted git repository. benedict pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/cassandra.git
The following commit(s) were added to refs/heads/trunk by this push: new adc3cdd Fix DecayingEstimatedHistogramReservoir Inefficiencies adc3cdd is described below commit adc3cdde2ae2a78f7d2bb66da47a07545d3e06cf Author: Jordan West <jorda...@gmail.com> AuthorDate: Wed Jan 8 11:09:08 2020 -0800 Fix DecayingEstimatedHistogramReservoir Inefficiencies Patch by Jordan West; reviewed by Benedict Elliott Smith for CASSANDRA-15213 --- .../DecayingEstimatedHistogramReservoir.java | 217 ++++++++++++++++----- .../DecayingEstimatedHistogramReservoirTest.java | 188 ++++++++++++++++-- 2 files changed, 338 insertions(+), 67 deletions(-) diff --git a/src/java/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoir.java b/src/java/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoir.java index a3168cf..60f381c 100644 --- a/src/java/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoir.java +++ b/src/java/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoir.java @@ -22,9 +22,8 @@ import java.io.OutputStream; import java.io.OutputStreamWriter; import java.io.PrintWriter; import java.nio.charset.StandardCharsets; -import java.util.Arrays; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.LongAdder; +import java.util.concurrent.atomic.AtomicLongArray; import com.google.common.annotations.VisibleForTesting; import com.google.common.primitives.Ints; @@ -34,6 +33,9 @@ import com.codahale.metrics.Reservoir; import com.codahale.metrics.Snapshot; import org.apache.cassandra.utils.EstimatedHistogram; +import static java.lang.Math.max; +import static java.lang.Math.min; + /** * A decaying histogram reservoir where values collected during each minute will be twice as significant as the values * collected in the previous minute. Measured values are collected in variable sized buckets, using small buckets in the @@ -63,30 +65,81 @@ import org.apache.cassandra.utils.EstimatedHistogram; * * Each bucket represents values from (previous bucket offset, current offset]. * + * To reduce contention each logical bucket is striped accross a configurable number of stripes (default: 4). Threads are + * assigned to specific stripes. In addition, logical buckets are distributed across the physical storage to reduce conention + * when logically adjacent buckets are updated. See CASSANDRA-15213. + * * [1]: http://dimacs.rutgers.edu/~graham/pubs/papers/fwddecay.pdf * [2]: https://en.wikipedia.org/wiki/Half-life * [3]: https://github.com/dropwizard/metrics/blob/v3.1.2/metrics-core/src/main/java/com/codahale/metrics/ExponentiallyDecayingReservoir.java */ public class DecayingEstimatedHistogramReservoir implements Reservoir { + /** * The default number of decayingBuckets. Use this bucket count to reduce memory allocation for bucket offsets. */ public static final int DEFAULT_BUCKET_COUNT = 164; + public static final int DEFAULT_STRIPE_COUNT = Integer.parseInt(System.getProperty("cassandra.dehr_stripe_count", "2")); + public static final int MAX_BUCKET_COUNT = 237; public static final boolean DEFAULT_ZERO_CONSIDERATION = false; + private static final int[] DISTRIBUTION_PRIMES = new int[] { 17, 19, 23, 29 }; + // The offsets used with a default sized bucket array without a separate bucket for zero values. public static final long[] DEFAULT_WITHOUT_ZERO_BUCKET_OFFSETS = EstimatedHistogram.newOffsets(DEFAULT_BUCKET_COUNT, false); // The offsets used with a default sized bucket array with a separate bucket for zero values. public static final long[] DEFAULT_WITH_ZERO_BUCKET_OFFSETS = EstimatedHistogram.newOffsets(DEFAULT_BUCKET_COUNT, true); + private static final int TABLE_BITS = 4; + private static final int TABLE_MASK = -1 >>> (32 - TABLE_BITS); + private static final float[] LOG2_TABLE = computeTable(TABLE_BITS); + private static final float log2_12_recp = (float) (1d / slowLog2(1.2d)); + + private static float[] computeTable(int bits) + { + float[] table = new float[1 << bits]; + for (int i = 1 ; i < 1<<bits ; ++i) + table[i] = (float) slowLog2(ratio(i, bits)); + return table; + } + + public static float fastLog12(long v) + { + return fastLog2(v) * log2_12_recp; + } + + // returns 0 for all inputs <= 1 + private static float fastLog2(long v) + { + v = max(v, 1); + int highestBitPosition = 63 - Long.numberOfLeadingZeros(v); + v = Long.rotateRight(v, highestBitPosition - TABLE_BITS); + int index = (int) (v & TABLE_MASK); + float result = LOG2_TABLE[index]; + result += highestBitPosition; + return result; + } + + private static double slowLog2(double v) + { + return Math.log(v) / Math.log(2); + } + + private static double ratio(int i, int bits) + { + return Float.intBitsToFloat((127 << 23) | (i << (23 - bits))); + } + // Represents the bucket offset as created by {@link EstimatedHistogram#newOffsets()} + private final int nStripes; private final long[] bucketOffsets; + private final int distributionPrime; // decayingBuckets and buckets are one element longer than bucketOffsets -- the last element is values greater than the last offset - private final LongAdder[] decayingBuckets; - private final LongAdder[] buckets; + private final AtomicLongArray decayingBuckets; + private final AtomicLongArray buckets; public static final long HALF_TIME_IN_S = 60L; public static final double MEAN_LIFETIME_IN_S = HALF_TIME_IN_S / Math.log(2.0); @@ -104,7 +157,7 @@ public class DecayingEstimatedHistogramReservoir implements Reservoir */ public DecayingEstimatedHistogramReservoir() { - this(DEFAULT_ZERO_CONSIDERATION, DEFAULT_BUCKET_COUNT, Clock.defaultClock()); + this(DEFAULT_ZERO_CONSIDERATION, DEFAULT_BUCKET_COUNT, DEFAULT_STRIPE_COUNT, Clock.defaultClock()); } /** @@ -115,7 +168,7 @@ public class DecayingEstimatedHistogramReservoir implements Reservoir */ public DecayingEstimatedHistogramReservoir(boolean considerZeroes) { - this(considerZeroes, DEFAULT_BUCKET_COUNT, Clock.defaultClock()); + this(considerZeroes, DEFAULT_BUCKET_COUNT, DEFAULT_STRIPE_COUNT, Clock.defaultClock()); } /** @@ -125,14 +178,22 @@ public class DecayingEstimatedHistogramReservoir implements Reservoir * same bucket as 1-value measurements * @param bucketCount number of buckets used to collect measured values */ - public DecayingEstimatedHistogramReservoir(boolean considerZeroes, int bucketCount) + public DecayingEstimatedHistogramReservoir(boolean considerZeroes, int bucketCount, int stripes) + { + this(considerZeroes, bucketCount, stripes, Clock.defaultClock()); + } + + @VisibleForTesting + public DecayingEstimatedHistogramReservoir(Clock clock) { - this(considerZeroes, bucketCount, Clock.defaultClock()); + this(DEFAULT_ZERO_CONSIDERATION, DEFAULT_BUCKET_COUNT, DEFAULT_STRIPE_COUNT, clock); } @VisibleForTesting - DecayingEstimatedHistogramReservoir(boolean considerZeroes, int bucketCount, Clock clock) + DecayingEstimatedHistogramReservoir(boolean considerZeroes, int bucketCount, int stripes, Clock clock) { + assert bucketCount <= MAX_BUCKET_COUNT : "bucket count cannot exceed: " + MAX_BUCKET_COUNT; + if (bucketCount == DEFAULT_BUCKET_COUNT) { if (considerZeroes == true) @@ -148,17 +209,22 @@ public class DecayingEstimatedHistogramReservoir implements Reservoir { bucketOffsets = EstimatedHistogram.newOffsets(bucketCount, considerZeroes); } - decayingBuckets = new LongAdder[bucketOffsets.length + 1]; - buckets = new LongAdder[bucketOffsets.length + 1]; - - for(int i = 0; i < buckets.length; i++) - { - decayingBuckets[i] = new LongAdder(); - buckets[i] = new LongAdder(); - } + nStripes = stripes; + decayingBuckets = new AtomicLongArray((bucketOffsets.length + 1) * nStripes); + buckets = new AtomicLongArray((bucketOffsets.length + 1) * nStripes); this.clock = clock; decayLandmark = clock.getTime(); + int distributionPrime = 1; + for (int prime : DISTRIBUTION_PRIMES) + { + if (buckets.length() % prime != 0) + { + distributionPrime = prime; + break; + } + } + this.distributionPrime = distributionPrime; } /** @@ -171,16 +237,44 @@ public class DecayingEstimatedHistogramReservoir implements Reservoir long now = clock.getTime(); rescaleIfNeeded(now); - int index = Arrays.binarySearch(bucketOffsets, value); - if (index < 0) - { - // inexact match, take the first bucket higher than n - index = -index - 1; - } - // else exact match; we're good + int index = findIndex(bucketOffsets, value); + + updateBucket(decayingBuckets, index, Math.round(forwardDecayWeight(now))); + updateBucket(buckets, index, 1); + } + + public void updateBucket(AtomicLongArray buckets, int index, long value) + { + int stripe = (int) (Thread.currentThread().getId() & (nStripes - 1)); + buckets.addAndGet(stripedIndex(index, stripe), value); + } + + public int stripedIndex(int offsetIndex, int stripe) + { + return (((offsetIndex * nStripes + stripe) * distributionPrime) % buckets.length()); + } + + @VisibleForTesting + public static int findIndex(long[] bucketOffsets, long value) + { + // values below zero are nonsense, but we have never failed when presented them + value = max(value, 0); - decayingBuckets[index].add(Math.round(forwardDecayWeight(now))); - buckets[index].increment(); + // The bucket index can be estimated using the equation Math.floor(Math.log(value) / Math.log(1.2)) + + // By using an integer domain we effectively squeeze multiple exponents of 1.2 into the same bucket, + // so for values > 2, we must "subtract" these exponents from the logarithm to determine which two buckets + // to consult (as our approximation otherwise produces a value that is within 1 of the true value) + int offset = (value > 2 ? 3 : 1) + (int)bucketOffsets[0]; + + // See DecayingEstimatedHistogramResevoirTest#showEstimationWorks and DecayingEstimatedHistogramResevoirTest#testFindIndex() + // for a runnable "proof" + // + // With this assumption, the estimate is calculated and the furthest offset from the estimation is checked + // if this bucket does not contain the value then the next one will + + int firstCandidate = max(0, min(bucketOffsets.length - 1, ((int) fastLog12(value)) - offset)); + return value <= bucketOffsets[firstCandidate] ? firstCandidate : firstCandidate + 1; } private double forwardDecayWeight(long now) @@ -189,17 +283,24 @@ public class DecayingEstimatedHistogramReservoir implements Reservoir } /** - * Return the number of buckets where recorded values are stored. + * Returns the logical number of buckets where recorded values are stored. The actual number of physical buckets + * is size() * stripeCount() * * This method does not return the number of recorded values as suggested by the {@link Reservoir} interface. * * @return the number of buckets + * @see #stripeCount() */ public int size() { - return decayingBuckets.length; + return bucketOffsets.length + 1; } + + public int stripeCount() + { + return nStripes; + } /** * Returns a snapshot of the decaying values in this reservoir. * @@ -219,7 +320,23 @@ public class DecayingEstimatedHistogramReservoir implements Reservoir @VisibleForTesting boolean isOverflowed() { - return decayingBuckets[decayingBuckets.length - 1].sum() > 0; + return bucketValue(bucketOffsets.length, true) > 0; + } + + private long bucketValue(int index, boolean withDecay) + { + long val = 0; + AtomicLongArray bs = withDecay ? decayingBuckets : buckets; + for (int stripe = 0; stripe < nStripes; stripe++) + val += bs.get(stripedIndex(index, stripe)); + + return val; + } + + @VisibleForTesting + long stripedBucketValue(int i, boolean withDecay) + { + return withDecay ? decayingBuckets.get(i) : buckets.get(i); } private void rescaleIfNeeded() @@ -248,14 +365,12 @@ public class DecayingEstimatedHistogramReservoir implements Reservoir private void rescale(long now) { + // despite striping its safe to rescale each bucket individually final double rescaleFactor = forwardDecayWeight(now); - - final int bucketCount = decayingBuckets.length; - for (int i = 0; i < bucketCount; i++) + for (int i = 0; i < decayingBuckets.length(); i++) { - long storedValue = decayingBuckets[i].sumThenReset(); - storedValue = Math.round(storedValue / rescaleFactor); - decayingBuckets[i].add(storedValue); + long newValue = Math.round(decayingBuckets.get(i) / rescaleFactor); + decayingBuckets.set(i, newValue); } } @@ -267,11 +382,11 @@ public class DecayingEstimatedHistogramReservoir implements Reservoir @VisibleForTesting public void clear() { - final int bucketCount = decayingBuckets.length; + final int bucketCount = decayingBuckets.length(); for (int i = 0; i < bucketCount; i++) { - decayingBuckets[i].reset(); - buckets[i].reset(); + decayingBuckets.set(i, 0L); + buckets.set(i, 0L); } } @@ -281,8 +396,8 @@ public class DecayingEstimatedHistogramReservoir implements Reservoir */ public void rebase(EstimatedHistogramReservoirSnapshot snapshot) { - // Check bucket count - if (decayingBuckets.length != snapshot.decayingBuckets.length) + // Check bucket count (a snapshot always has one stripe so the logical bucket count is used + if (size() != snapshot.decayingBuckets.length) { throw new IllegalStateException("Unable to merge two DecayingEstimatedHistogramReservoirs with different bucket sizes"); } @@ -297,14 +412,18 @@ public class DecayingEstimatedHistogramReservoir implements Reservoir } this.decayLandmark = snapshot.snapshotLandmark; - for (int i = 0; i < decayingBuckets.length; i++) + for (int i = 0; i < size(); i++) { - decayingBuckets[i].reset(); - buckets[i].reset(); - - decayingBuckets[i].add(snapshot.decayingBuckets[i]); - buckets[i].add(snapshot.values[i]); + // set rebased values in the first stripe and clear out all other data + decayingBuckets.set(stripedIndex(i, 0), snapshot.decayingBuckets[i]); + buckets.set(stripedIndex(i, 0), snapshot.values[i]); + for (int stripe = 1; stripe < nStripes; stripe++) + { + decayingBuckets.set(stripedIndex(i, stripe), 0); + buckets.set(stripedIndex(i, stripe), 0); + } } + } /** @@ -328,7 +447,7 @@ public class DecayingEstimatedHistogramReservoir implements Reservoir public EstimatedHistogramReservoirSnapshot(DecayingEstimatedHistogramReservoir reservoir) { - final int length = reservoir.decayingBuckets.length; + final int length = reservoir.size(); final double rescaleFactor = reservoir.forwardDecayWeight(reservoir.clock.getTime()); this.decayingBuckets = new long[length]; @@ -338,8 +457,8 @@ public class DecayingEstimatedHistogramReservoir implements Reservoir for (int i = 0; i < length; i++) { - this.decayingBuckets[i] = Math.round(reservoir.decayingBuckets[i].sum() / rescaleFactor); - this.values[i] = reservoir.buckets[i].sum(); + this.decayingBuckets[i] = Math.round(reservoir.bucketValue(i, true) / rescaleFactor); + this.values[i] = reservoir.bucketValue(i, false); } this.count = count(); this.reservoir = reservoir; diff --git a/test/unit/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoirTest.java b/test/unit/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoirTest.java index 4a9d18b..e4289f7 100644 --- a/test/unit/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoirTest.java +++ b/test/unit/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoirTest.java @@ -18,20 +18,166 @@ package org.apache.cassandra.metrics; +import java.util.Arrays; +import java.util.BitSet; +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.LockSupport; + +import org.junit.Assert; +import org.junit.Ignore; import org.junit.Test; import com.codahale.metrics.Clock; import com.codahale.metrics.Snapshot; +import org.apache.cassandra.utils.EstimatedHistogram; +import org.apache.cassandra.utils.Pair; +import org.quicktheories.core.Gen; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; - +import static org.quicktheories.QuickTheory.qt; +import static org.quicktheories.generators.SourceDSL.*; public class DecayingEstimatedHistogramReservoirTest { private static final double DOUBLE_ASSERT_DELTA = 0; + public static final int numExamples = 1000000; + public static final Gen<long[]> offsets = integers().from(DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT) + .upToAndIncluding(DecayingEstimatedHistogramReservoir.MAX_BUCKET_COUNT - 10) + .zip(booleans().all(), EstimatedHistogram::newOffsets); + + + @Test + public void testFindIndex() + { + qt().withExamples(numExamples) + .forAll(booleans().all() + .flatMap(b -> offsets.flatMap(offs -> this.offsetsAndValue(offs, b, 0)))) + .check(this::checkFindIndex); + } + + private boolean checkFindIndex(Pair<long[], Long> offsetsAndValue) + { + long[] offsets = offsetsAndValue.left; + long value = offsetsAndValue.right; + + int model = findIndexModel(offsets, value); + int actual = DecayingEstimatedHistogramReservoir.findIndex(offsets, value); + + return model == actual; + } + + private int findIndexModel(long[] offsets, long value) + { + int modelIndex = Arrays.binarySearch(offsets, value); + if (modelIndex < 0) + modelIndex = -modelIndex - 1; + + return modelIndex; + }; + + @Test + public void showEstimationWorks() + { + qt().withExamples(numExamples) + .forAll(offsets.flatMap(offs -> this.offsetsAndValue(offs, false, 9))) + .check(this::checkEstimation); + } + + public boolean checkEstimation(Pair<long[], Long> offsetsAndValue) + { + long[] offsets = offsetsAndValue.left; + long value = offsetsAndValue.right; + boolean considerZeros = offsets[0] == 0; + + int modelIndex = Arrays.binarySearch(offsets, value); + if (modelIndex < 0) + modelIndex = -modelIndex - 1; + + int estimate = (int) DecayingEstimatedHistogramReservoir.fastLog12(value); + + if (considerZeros) + return estimate - 3 == modelIndex || estimate - 2 == modelIndex; + else + return estimate - 4 == modelIndex || estimate - 3 == modelIndex; + } + + + private Gen<Pair<long[], Long>> offsetsAndValue(long[] offsets, boolean useMaxLong, long minValue) + { + return longs().between(minValue, useMaxLong ? Long.MAX_VALUE : offsets[offsets.length - 1] + 100) + .mix(longs().between(minValue, minValue + 10),50) + .map(value -> Pair.create(offsets, value)); + } + + //shows that the max before overflow is 238 buckets regardless of consider zeros + @Test + @Ignore + public void showHistorgramOffsetOverflow() + { + qt().forAll(integers().from(DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT).upToAndIncluding(1000)) + .check(count -> { + long[] offsets = EstimatedHistogram.newOffsets(count, false); + for (long offset : offsets) + if (offset < 0) + return false; + + return true; + }); + } + + @Test + public void testStriping() throws InterruptedException + { + TestClock clock = new TestClock(); + int nStripes = 4; + DecayingEstimatedHistogramReservoir model = new DecayingEstimatedHistogramReservoir(clock); + DecayingEstimatedHistogramReservoir test = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, + DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, + nStripes, + clock); + + long seed = System.nanoTime(); + System.out.println("DecayingEstimatedHistogramReservoirTest#testStriping.seed = " + seed); + Random valGen = new Random(seed); + ExecutorService executors = Executors.newFixedThreadPool(nStripes * 2); + for (int i = 0; i < 1_000_000; i++) + { + long value = Math.abs(valGen.nextInt()); + executors.submit(() -> { + model.update(value); + LockSupport.parkNanos(2); + test.update(value); + }); + } + + executors.shutdown(); + executors.awaitTermination(10, TimeUnit.SECONDS); + + Snapshot modelSnapshot = model.getSnapshot(); + Snapshot testSnapshot = test.getSnapshot(); + + assertEquals(modelSnapshot.getMean(), testSnapshot.getMean(), DOUBLE_ASSERT_DELTA); + assertEquals(modelSnapshot.getMin(), testSnapshot.getMin(), DOUBLE_ASSERT_DELTA); + assertEquals(modelSnapshot.getMax(), testSnapshot.getMax(), DOUBLE_ASSERT_DELTA); + assertEquals(modelSnapshot.getMedian(), testSnapshot.getMedian(), DOUBLE_ASSERT_DELTA); + for (double i = 0.0; i < 1.0; i += 0.1) + assertEquals(modelSnapshot.getValue(i), testSnapshot.getValue(i), DOUBLE_ASSERT_DELTA); + + + int stripedValues = 0; + for (int i = model.size(); i < model.size() * model.stripeCount(); i++) + { + stripedValues += model.stripedBucketValue(i, true); + } + assertTrue("no striping found", stripedValues > 0); + } + @Test public void testSimple() { @@ -45,7 +191,7 @@ public class DecayingEstimatedHistogramReservoirTest } { // 0 and 1 map to different buckets - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(true, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(true); histogram.update(0); assertEquals(1, histogram.getSnapshot().getValues()[0]); histogram.update(1); @@ -58,7 +204,7 @@ public class DecayingEstimatedHistogramReservoirTest @Test public void testOverflow() { - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, 1); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, 1, 1); histogram.update(100); assert histogram.isOverflowed(); assertEquals(Long.MAX_VALUE, histogram.getSnapshot().getMax()); @@ -80,7 +226,7 @@ public class DecayingEstimatedHistogramReservoirTest { TestClock clock = new TestClock(); - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(clock); for (int i = 0; i < 40; i++) histogram.update(0); for (int i = 0; i < 20; i++) @@ -92,7 +238,10 @@ public class DecayingEstimatedHistogramReservoirTest { TestClock clock = new TestClock(); - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(true, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(true, + DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, + DecayingEstimatedHistogramReservoir.DEFAULT_STRIPE_COUNT, + clock); for (int i = 0; i < 40; i++) histogram.update(0); for (int i = 0; i < 20; i++) @@ -109,7 +258,7 @@ public class DecayingEstimatedHistogramReservoirTest { TestClock clock = new TestClock(); - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(clock); for (int i = 0; i < 20; i++) histogram.update(10); for (int i = 0; i < 40; i++) @@ -128,7 +277,7 @@ public class DecayingEstimatedHistogramReservoirTest { TestClock clock = new TestClock(); - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, 90, clock); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, 90, 1, clock); histogram.update(23282687); assertFalse(histogram.isOverflowed()); assertEquals(1, histogram.getSnapshot().getValues()[89]); @@ -149,7 +298,7 @@ public class DecayingEstimatedHistogramReservoirTest { TestClock clock = new TestClock(); - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(clock); // percentile of empty histogram is 0 assertEquals(0D, histogram.getSnapshot().getValue(0.99), DOUBLE_ASSERT_DELTA); @@ -164,7 +313,7 @@ public class DecayingEstimatedHistogramReservoirTest { TestClock clock = new TestClock(); - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(clock); histogram.update(1); histogram.update(2); @@ -182,7 +331,7 @@ public class DecayingEstimatedHistogramReservoirTest { TestClock clock = new TestClock(); - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(clock); for (int i = 11; i <= 20; i++) histogram.update(i); @@ -201,7 +350,10 @@ public class DecayingEstimatedHistogramReservoirTest { TestClock clock = new TestClock(); - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(true, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(true, + DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, + DecayingEstimatedHistogramReservoir.DEFAULT_STRIPE_COUNT, + clock); histogram.update(0); histogram.update(0); histogram.update(1); @@ -219,7 +371,7 @@ public class DecayingEstimatedHistogramReservoirTest { TestClock clock = new TestClock(); - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(clock); // percentile of empty histogram is 0 assertEquals(0, histogram.getSnapshot().getValue(1.0), DOUBLE_ASSERT_DELTA); @@ -312,7 +464,7 @@ public class DecayingEstimatedHistogramReservoirTest { TestClock clock = new TestClock(); - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(clock); // percentile of empty histogram is 0 assertEquals(0, histogram.getSnapshot().getValue(0.99), DOUBLE_ASSERT_DELTA); @@ -334,7 +486,7 @@ public class DecayingEstimatedHistogramReservoirTest { TestClock clock = new TestClock(); - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(clock); histogram.update(20); histogram.update(21); @@ -360,7 +512,7 @@ public class DecayingEstimatedHistogramReservoirTest { TestClock clock = new TestClock(); - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(clock); clock.addMillis(DecayingEstimatedHistogramReservoir.LANDMARK_RESET_INTERVAL_IN_MS - 1_000L); @@ -385,8 +537,8 @@ public class DecayingEstimatedHistogramReservoirTest { TestClock clock = new TestClock(); - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock); - DecayingEstimatedHistogramReservoir another = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(clock); + DecayingEstimatedHistogramReservoir another = new DecayingEstimatedHistogramReservoir(clock); clock.addMillis(DecayingEstimatedHistogramReservoir.LANDMARK_RESET_INTERVAL_IN_MS - 1_000L); @@ -420,7 +572,7 @@ public class DecayingEstimatedHistogramReservoirTest { TestClock clock = new TestClock(); - DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock); + DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(clock); histogram.update(42); histogram.update(42); assertEquals(2, histogram.getSnapshot().size()); --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org