You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by be...@apache.org on 2020/02/19 11:33:06 UTC

[cassandra] branch trunk updated: Fix DecayingEstimatedHistogramReservoir Inefficiencies

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 <jo...@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-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org