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 2014/12/12 14:45:26 UTC

cassandra git commit: Reduce cassandra-stress sampling memory requirements

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 7e3f6151a -> cdba5aac9


 Reduce cassandra-stress sampling memory requirements

patch by benedict; reviewed by branimir for CASSANDRA-7926


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/cdba5aac
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/cdba5aac
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/cdba5aac

Branch: refs/heads/cassandra-2.1
Commit: cdba5aac9b0166bef0474e5cf8aced010762992a
Parents: 7e3f615
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Fri Dec 12 13:44:34 2014 +0000
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Fri Dec 12 13:44:34 2014 +0000

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/stress/StressAction.java   | 11 ++-
 .../apache/cassandra/stress/StressMetrics.java  |  4 +-
 .../cassandra/stress/settings/CliOption.java    |  1 +
 .../stress/settings/SettingsCommand.java        |  4 +-
 .../stress/settings/SettingsSamples.java        | 94 ++++++++++++++++++++
 .../stress/settings/StressSettings.java         |  7 +-
 .../cassandra/stress/util/SampleOfLongs.java    | 59 +++++-------
 .../org/apache/cassandra/stress/util/Timer.java | 28 +++---
 .../apache/cassandra/stress/util/Timing.java    | 26 +++---
 .../cassandra/stress/util/TimingInterval.java   |  4 +-
 11 files changed, 168 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/cdba5aac/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 18efc7e..fa42e85 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.3
+ * Reduce cassandra-stress sampling memory requirements (CASSANDRA-7926)
  * Ensure memtable flush cannot expire commit log entries from its future (CASSANDRA-8383)
  * Make read "defrag" async to reclaim memtables (CASSANDRA-8459)
  * Remove tmplink files for offline compactions (CASSANDRA-8321)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cdba5aac/tools/stress/src/org/apache/cassandra/stress/StressAction.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/StressAction.java b/tools/stress/src/org/apache/cassandra/stress/StressAction.java
index b50637f..68e0004 100644
--- a/tools/stress/src/org/apache/cassandra/stress/StressAction.java
+++ b/tools/stress/src/org/apache/cassandra/stress/StressAction.java
@@ -189,7 +189,10 @@ public class StressAction implements Runnable
         final CountDownLatch done = new CountDownLatch(threadCount);
         final Consumer[] consumers = new Consumer[threadCount];
         for (int i = 0; i < threadCount; i++)
-            consumers[i] = new Consumer(operations, done, workManager, metrics, rateLimiter);
+        {
+            Timer timer = metrics.getTiming().newTimer(settings.samples.liveCount / threadCount);
+            consumers[i] = new Consumer(operations, done, workManager, timer, metrics, rateLimiter);
+        }
 
         // starting worker threadCount
         for (int i = 0; i < threadCount; i++)
@@ -246,19 +249,19 @@ public class StressAction implements Runnable
         private final WorkManager workManager;
         private final CountDownLatch done;
 
-        public Consumer(OpDistributionFactory operations, CountDownLatch done, WorkManager workManager, StressMetrics metrics, RateLimiter rateLimiter)
+        public Consumer(OpDistributionFactory operations, CountDownLatch done, WorkManager workManager, Timer timer, StressMetrics metrics, RateLimiter rateLimiter)
         {
             this.done = done;
             this.rateLimiter = rateLimiter;
             this.workManager = workManager;
             this.metrics = metrics;
-            this.timer = metrics.getTiming().newTimer();
+            this.timer = timer;
             this.operations = operations.get(timer);
         }
 
         public void run()
         {
-
+            timer.init();
             try
             {
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cdba5aac/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java b/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java
index 00f479e..3a4a4a3 100644
--- a/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java
+++ b/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java
@@ -27,7 +27,6 @@ import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.lang3.time.DurationFormatUtils;
 
@@ -50,7 +49,7 @@ public class StressMetrics
     private volatile boolean cancelled = false;
     private final Uncertainty rowRateUncertainty = new Uncertainty();
     private final CountDownLatch stopped = new CountDownLatch(1);
-    private final Timing timing = new Timing();
+    private final Timing timing;
     private final Callable<JmxCollector.GcStats> gcStatsCollector;
     private volatile JmxCollector.GcStats totalGcStats;
 
@@ -80,6 +79,7 @@ public class StressMetrics
             };
         }
         this.gcStatsCollector = gcStatsCollector;
+        this.timing = new Timing(settings.samples.historyCount, settings.samples.reportCount);
 
         printHeader("", output);
         thread = tf.newThread(new Runnable()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cdba5aac/tools/stress/src/org/apache/cassandra/stress/settings/CliOption.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/CliOption.java b/tools/stress/src/org/apache/cassandra/stress/settings/CliOption.java
index 7fd10ff..5ec56dc 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/CliOption.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/CliOption.java
@@ -32,6 +32,7 @@ public enum CliOption
     RATE("Thread count, rate limit or automatic mode (default is auto)", SettingsRate.helpPrinter()),
     MODE("Thrift or CQL with options", SettingsMode.helpPrinter()),
     ERRORS("How to handle errors when encountered during stress", SettingsErrors.helpPrinter()),
+    SAMPLE("Specify the number of samples to collect for measuring latency", SettingsSamples.helpPrinter()),
     SCHEMA("Replication settings, compression, compaction, etc.", SettingsSchema.helpPrinter()),
     NODE("Nodes to connect to", SettingsNode.helpPrinter()),
     LOG("Where to log progress to, and the interval at which to do it", SettingsLog.helpPrinter()),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cdba5aac/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java
index a1c89e1..8850544 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java
@@ -63,7 +63,7 @@ public abstract class SettingsCommand implements Serializable
         this.noWarmup = options.noWarmup.setByUser();
         if (count != null)
         {
-            this.count = Long.parseLong(count.count.value());
+            this.count = OptionDistribution.parseLong(count.count.value());
             this.duration = 0;
             this.durationUnits = null;
             this.targetUncertainty = -1;
@@ -113,7 +113,7 @@ public abstract class SettingsCommand implements Serializable
 
     static class Count extends Options
     {
-        final OptionSimple count = new OptionSimple("n=", "[0-9]+", null, "Number of operations to perform", true);
+        final OptionSimple count = new OptionSimple("n=", "[0-9]+[bmk]?", null, "Number of operations to perform", true);
         @Override
         public List<? extends Option> options()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cdba5aac/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSamples.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSamples.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSamples.java
new file mode 100644
index 0000000..7a9f484
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSamples.java
@@ -0,0 +1,94 @@
+package org.apache.cassandra.stress.settings;
+/*
+ * 
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+public class SettingsSamples implements Serializable
+{
+
+    public final int liveCount;
+    public final int historyCount;
+    public final int reportCount;
+
+    public SettingsSamples(SampleOptions options)
+    {
+        liveCount = (int) OptionDistribution.parseLong(options.liveCount.value());
+        historyCount = (int) OptionDistribution.parseLong(options.historyCount.value());
+        reportCount = (int) OptionDistribution.parseLong(options.reportCount.value());
+    }
+
+    // Option Declarations
+
+    private static final class SampleOptions extends GroupedOptions
+    {
+        final OptionSimple historyCount = new OptionSimple("history=", "[0-9]+[bmk]?", "50K", "The number of samples to save across the whole run", false);
+        final OptionSimple liveCount = new OptionSimple("live=", "[0-9]+[bmk]?", "1M", "The number of samples to save between reports", false);
+        final OptionSimple reportCount = new OptionSimple("report=", "[0-9]+[bmk]?", "100K", "The maximum number of samples to use when building a report", false);
+
+        @Override
+        public List<? extends Option> options()
+        {
+            return Arrays.asList(historyCount, liveCount, reportCount);
+        }
+    }
+
+    // CLI Utility Methods
+
+    public static SettingsSamples get(Map<String, String[]> clArgs)
+    {
+        String[] params = clArgs.remove("-sample");
+        if (params == null)
+        {
+            return new SettingsSamples(new SampleOptions());
+        }
+        SampleOptions options = GroupedOptions.select(params, new SampleOptions());
+        if (options == null)
+        {
+            printHelp();
+            System.out.println("Invalid -sample options provided, see output for valid options");
+            System.exit(1);
+        }
+        return new SettingsSamples(options);
+    }
+
+    public static void printHelp()
+    {
+        GroupedOptions.printOptions(System.out, "-sample", new SampleOptions());
+    }
+
+    public static Runnable helpPrinter()
+    {
+        return new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                printHelp();
+            }
+        };
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cdba5aac/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java b/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java
index ba72821..815fe9b 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java
@@ -46,6 +46,7 @@ public class StressSettings implements Serializable
     public final SettingsPopulation generate;
     public final SettingsInsert insert;
     public final SettingsColumn columns;
+    public final SettingsSamples samples;
     public final SettingsErrors errors;
     public final SettingsLog log;
     public final SettingsMode mode;
@@ -55,13 +56,14 @@ public class StressSettings implements Serializable
     public final SettingsPort port;
     public final String sendToDaemon;
 
-    public StressSettings(SettingsCommand command, SettingsRate rate, SettingsPopulation generate, SettingsInsert insert, SettingsColumn columns, SettingsErrors errors, SettingsLog log, SettingsMode mode, SettingsNode node, SettingsSchema schema, SettingsTransport transport, SettingsPort port, String sendToDaemon)
+    public StressSettings(SettingsCommand command, SettingsRate rate, SettingsPopulation generate, SettingsInsert insert, SettingsColumn columns, SettingsSamples samples, SettingsErrors errors, SettingsLog log, SettingsMode mode, SettingsNode node, SettingsSchema schema, SettingsTransport transport, SettingsPort port, String sendToDaemon)
     {
         this.command = command;
         this.rate = rate;
         this.insert = insert;
         this.generate = generate;
         this.columns = columns;
+        this.samples = samples;
         this.errors = errors;
         this.log = log;
         this.mode = mode;
@@ -232,6 +234,7 @@ public class StressSettings implements Serializable
         SettingsPopulation generate = SettingsPopulation.get(clArgs, command);
         SettingsInsert insert = SettingsInsert.get(clArgs);
         SettingsColumn columns = SettingsColumn.get(clArgs);
+        SettingsSamples samples = SettingsSamples.get(clArgs);
         SettingsErrors errors = SettingsErrors.get(clArgs);
         SettingsLog log = SettingsLog.get(clArgs);
         SettingsMode mode = SettingsMode.get(clArgs);
@@ -254,7 +257,7 @@ public class StressSettings implements Serializable
             }
             System.exit(1);
         }
-        return new StressSettings(command, rate, generate, insert, columns, errors, log, mode, node, schema, transport, port, sendToDaemon);
+        return new StressSettings(command, rate, generate, insert, columns, samples, errors, log, mode, node, schema, transport, port, sendToDaemon);
     }
 
     private static Map<String, String[]> parseMap(String[] args)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cdba5aac/tools/stress/src/org/apache/cassandra/stress/util/SampleOfLongs.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/SampleOfLongs.java b/tools/stress/src/org/apache/cassandra/stress/util/SampleOfLongs.java
index 8be9f81..ed54ee0 100644
--- a/tools/stress/src/org/apache/cassandra/stress/util/SampleOfLongs.java
+++ b/tools/stress/src/org/apache/cassandra/stress/util/SampleOfLongs.java
@@ -51,61 +51,44 @@ public final class SampleOfLongs
 
     static SampleOfLongs merge(Random rnd, List<SampleOfLongs> merge, int maxSamples)
     {
-        int maxLength = 0;
+        // grab the lowest probability of selection, and normalise all samples to that
         double targetp = 1;
         for (SampleOfLongs sampleOfLongs : merge)
-        {
-            maxLength += sampleOfLongs.sample.length;
             targetp = Math.min(targetp, sampleOfLongs.p);
+
+        // calculate how many samples we should encounter
+        int maxLength = 0;
+        for (SampleOfLongs sampleOfLongs : merge)
+            maxLength += sampleOfLongs.sample.length * (targetp / sampleOfLongs.p);
+
+        if (maxLength > maxSamples)
+        {
+            targetp *= maxSamples / (double) maxLength;
+            maxLength = maxSamples;
         }
+
         long[] sample = new long[maxLength];
         int count = 0;
-        for (SampleOfLongs latencies : merge)
+        out: for (SampleOfLongs latencies : merge)
         {
             long[] in = latencies.sample;
             double p = targetp / latencies.p;
             for (int i = 0 ; i < in.length ; i++)
+            {
                 if (rnd.nextDouble() < p)
+                {
                     sample[count++] = in[i];
+                    if (count == maxLength)
+                        break out;
+                }
+            }
         }
-        if (count > maxSamples)
-        {
-            targetp = subsample(rnd, maxSamples, sample, count, targetp);
-            count = maxSamples;
-        }
-        sample = Arrays.copyOf(sample, count);
+        if (count != maxLength)
+            sample = Arrays.copyOf(sample, count);
         Arrays.sort(sample);
         return new SampleOfLongs(sample, targetp);
     }
 
-    public SampleOfLongs subsample(Random rnd, int maxSamples)
-    {
-        if (maxSamples > sample.length)
-            return this;
-
-        long[] sample = this.sample.clone();
-        double p = subsample(rnd, maxSamples, sample, sample.length, this.p);
-        sample = Arrays.copyOf(sample, maxSamples);
-        return new SampleOfLongs(sample, p);
-    }
-
-    private static double subsample(Random rnd, int maxSamples, long[] sample, int count, double p)
-    {
-        // want exactly maxSamples, so select random indexes up to maxSamples
-        for (int i = 0 ; i < maxSamples ; i++)
-        {
-            int take = i + rnd.nextInt(count - i);
-            long tmp = sample[i];
-            sample[i] = sample[take];
-            sample[take] = tmp;
-        }
-
-        // calculate new p; have selected with probability maxSamples / count
-        // so multiply p by this probability
-        p *= maxSamples / (double) sample.length;
-        return p;
-    }
-
     public double medianLatency()
     {
         if (sample.length == 0)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cdba5aac/tools/stress/src/org/apache/cassandra/stress/util/Timer.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/Timer.java b/tools/stress/src/org/apache/cassandra/stress/util/Timer.java
index 4e2b0a3..7449a00 100644
--- a/tools/stress/src/org/apache/cassandra/stress/util/Timer.java
+++ b/tools/stress/src/org/apache/cassandra/stress/util/Timer.java
@@ -25,21 +25,18 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Random;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
 
 // a timer - this timer must be used by a single thread, and co-ordinates with other timers by
 public final class Timer
 {
-
-    private static final int SAMPLE_SIZE_SHIFT = 14;
-    private static final int SAMPLE_SIZE_MASK = (1 << SAMPLE_SIZE_SHIFT) - 1;
-
-    private final Random rnd = new Random();
+    private ThreadLocalRandom rnd;
 
     // in progress snap start
     private long sampleStartNanos;
 
     // each entry is present with probability 1/p(opCount) or 1/(p(opCount)-1)
-    private final long[] sample = new long[1 << SAMPLE_SIZE_SHIFT];
+    private final long[] sample;
     private int opCount;
 
     // aggregate info
@@ -56,14 +53,25 @@ public final class Timer
     volatile TimingInterval report;
     private volatile TimingInterval finalReport;
 
+    public Timer(int sampleCount)
+    {
+        int powerOf2 = 32 - Integer.numberOfLeadingZeros(sampleCount - 1);
+        this.sample = new long[1 << powerOf2];
+    }
+
+    public void init()
+    {
+        rnd = ThreadLocalRandom.current();
+    }
+
     public void start(){
         // decide if we're logging this event
         sampleStartNanos = System.nanoTime();
     }
 
-    private static int p(int index)
+    private int p(int index)
     {
-        return 1 + (index >>> SAMPLE_SIZE_SHIFT);
+        return 1 + (index / sample.length);
     }
 
     public boolean running()
@@ -90,9 +98,9 @@ public final class Timer
         upToDateAsOf = now;
     }
 
-    private static int index(int count)
+    private int index(int count)
     {
-        return count & SAMPLE_SIZE_MASK;
+        return count & (sample.length - 1);
     }
 
     private TimingInterval buildReport()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cdba5aac/tools/stress/src/org/apache/cassandra/stress/util/Timing.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/Timing.java b/tools/stress/src/org/apache/cassandra/stress/util/Timing.java
index 9eadf42..f48ce35 100644
--- a/tools/stress/src/org/apache/cassandra/stress/util/Timing.java
+++ b/tools/stress/src/org/apache/cassandra/stress/util/Timing.java
@@ -40,9 +40,16 @@ public class Timing
 
     private final CopyOnWriteArrayList<Timer> timers = new CopyOnWriteArrayList<>();
     private volatile TimingInterval history;
-    private final Random rnd = new Random();
+    private final int historySampleCount;
+    private final int reportSampleCount;
     private boolean done;
 
+    public Timing(int historySampleCount, int reportSampleCount)
+    {
+        this.historySampleCount = historySampleCount;
+        this.reportSampleCount = reportSampleCount;
+    }
+
     // TIMING
 
     public static class TimingResult<E>
@@ -56,7 +63,7 @@ public class Timing
         }
     }
 
-    private <E> TimingResult<E> snap(Random rnd, Callable<E> call) throws InterruptedException
+    public <E> TimingResult<E> snap(Callable<E> call) throws InterruptedException
     {
         final Timer[] timers = this.timers.toArray(new Timer[0]);
         final CountDownLatch ready = new CountDownLatch(timers.length);
@@ -92,13 +99,15 @@ public class Timing
         }
 
         this.done = done;
-        return new TimingResult<>(extra, TimingInterval.merge(rnd, intervals, Integer.MAX_VALUE, history.endNanos()));
+        TimingResult<E> result = new TimingResult<>(extra, TimingInterval.merge(intervals, reportSampleCount, history.endNanos()));
+        history = TimingInterval.merge(Arrays.asList(result.timing, history), historySampleCount, history.startNanos());
+        return result;
     }
 
     // build a new timer and add it to the set of running timers
-    public Timer newTimer()
+    public Timer newTimer(int sampleCount)
     {
-        final Timer timer = new Timer();
+        final Timer timer = new Timer(sampleCount);
         timers.add(timer);
         return timer;
     }
@@ -113,13 +122,6 @@ public class Timing
         return done;
     }
 
-    public <E> TimingResult<E> snap(Callable<E> call) throws InterruptedException
-    {
-        final TimingResult<E> result = snap(rnd, call);
-        history = TimingInterval.merge(rnd, Arrays.asList(result.timing, history), 200000, history.startNanos());
-        return result;
-    }
-
     public TimingInterval getHistory()
     {
         return history;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cdba5aac/tools/stress/src/org/apache/cassandra/stress/util/TimingInterval.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/TimingInterval.java b/tools/stress/src/org/apache/cassandra/stress/util/TimingInterval.java
index 50ab608..11d313f 100644
--- a/tools/stress/src/org/apache/cassandra/stress/util/TimingInterval.java
+++ b/tools/stress/src/org/apache/cassandra/stress/util/TimingInterval.java
@@ -25,6 +25,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
 
 // represents measurements taken over an interval of time
 // used for both single timer results and merged timer results
@@ -68,8 +69,9 @@ public final class TimingInterval
     }
 
     // merge multiple timer intervals together
-    static TimingInterval merge(Random rnd, List<TimingInterval> intervals, int maxSamples, long start)
+    static TimingInterval merge(List<TimingInterval> intervals, int maxSamples, long start)
     {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
         long operationCount = 0, partitionCount = 0, rowCount = 0;
         long maxLatency = 0, totalLatency = 0;
         List<SampleOfLongs> latencies = new ArrayList<>();