You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by up...@apache.org on 2018/11/27 20:18:36 UTC

[geode-benchmarks] branch develop updated: GEODE-6081: Probe that uses HdrHistogram

This is an automated email from the ASF dual-hosted git repository.

upthewaterspout pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode-benchmarks.git


The following commit(s) were added to refs/heads/develop by this push:
     new ab86586  GEODE-6081: Probe that uses HdrHistogram
ab86586 is described below

commit ab86586d09f281afaa8a4f42f98b929843ad2f82
Author: Dan Smith <up...@apache.org>
AuthorDate: Mon Nov 19 16:15:48 2018 -0800

    GEODE-6081: Probe that uses HdrHistogram
    
    Adding a new probe that uses HdrHistogram to generate a summary with a
    mean and 99th percentile latency.
    
    HdrHistogram should be more accurate than yardsticks histogram probe.
    
    This probe currently just reports a one line summary, but we could
    consider replacing the existing percentile and throughtput probes with
    this thing in the future - it has its own file format for recording per
    sample histograms.
---
 harness/build.gradle                               |   2 +
 .../apache/geode/perftest/yardstick/Clock.java}    |  22 +---
 .../perftest/yardstick/HdrHistogramProbe.java      | 115 +++++++++++++++++++++
 .../geode/perftest/yardstick/YardstickTask.java    |   2 +-
 .../perftest/yardstick/HdrHistogramProbeTest.java  | 103 ++++++++++++++++++
 5 files changed, 224 insertions(+), 20 deletions(-)

diff --git a/harness/build.gradle b/harness/build.gradle
index 7b32cd7..7fc72e9 100644
--- a/harness/build.gradle
+++ b/harness/build.gradle
@@ -33,8 +33,10 @@ dependencies {
     compile group: 'com.hierynomus', name: 'sshj', version: '0.26.0'
     compile group: 'commons-io', name: 'commons-io', version: '2.6'
     compile group: 'org.yardstickframework', name: 'yardstick', version: '0.8.3'
+    compile group: 'org.hdrhistogram', name: 'HdrHistogram', version: '2.1.10'
     testCompile group: 'org.mockito', name: 'mockito-all', version: '1.10.19'
     testCompile group: 'org.awaitility', name: 'awaitility', version: '3.0.0'
     testCompile group: 'org.slf4j', name: 'slf4j-simple', version: '1.7.25'
     testCompile group: 'org.apache.sshd', name: 'sshd-core', version: '2.1.0'
+    testCompile group: 'org.assertj', name: 'assertj-core', version: '3.11.1'
 }
diff --git a/harness/build.gradle b/harness/src/main/java/org/apache/geode/perftest/yardstick/Clock.java
similarity index 51%
copy from harness/build.gradle
copy to harness/src/main/java/org/apache/geode/perftest/yardstick/Clock.java
index 7b32cd7..462bbb7 100644
--- a/harness/build.gradle
+++ b/harness/src/main/java/org/apache/geode/perftest/yardstick/Clock.java
@@ -15,26 +15,10 @@
  * limitations under the License.
  */
 
-plugins {
-    id 'java'
-}
-
-group 'org.upthewaterspout.geode-performance'
-version '1.0-SNAPSHOT'
+package org.apache.geode.perftest.yardstick;
 
-sourceCompatibility = 1.8
+public interface Clock {
 
-repositories {
-    mavenCentral()
-}
+  long currentTimeNanos();
 
-dependencies {
-    testCompile group: 'junit', name: 'junit', version: '4.12'
-    compile group: 'com.hierynomus', name: 'sshj', version: '0.26.0'
-    compile group: 'commons-io', name: 'commons-io', version: '2.6'
-    compile group: 'org.yardstickframework', name: 'yardstick', version: '0.8.3'
-    testCompile group: 'org.mockito', name: 'mockito-all', version: '1.10.19'
-    testCompile group: 'org.awaitility', name: 'awaitility', version: '3.0.0'
-    testCompile group: 'org.slf4j', name: 'slf4j-simple', version: '1.7.25'
-    testCompile group: 'org.apache.sshd', name: 'sshd-core', version: '2.1.0'
 }
diff --git a/harness/src/main/java/org/apache/geode/perftest/yardstick/HdrHistogramProbe.java b/harness/src/main/java/org/apache/geode/perftest/yardstick/HdrHistogramProbe.java
new file mode 100644
index 0000000..7a7e79e
--- /dev/null
+++ b/harness/src/main/java/org/apache/geode/perftest/yardstick/HdrHistogramProbe.java
@@ -0,0 +1,115 @@
+package org.apache.geode.perftest.yardstick;
+
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.IntStream;
+
+import org.HdrHistogram.Histogram;
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkDriver;
+import org.yardstickframework.BenchmarkExecutionAwareProbe;
+import org.yardstickframework.BenchmarkProbePoint;
+import org.yardstickframework.BenchmarkTotalsOnlyProbe;
+
+/**
+ * Probe which returns generates throughput and latency information
+ * using HdrHistogram.
+ *
+ * This probe currently just reports a single summary of the histogram
+ * at the end of the test. HdrHistogram actually has a lot of nice support for
+ * recording histogram values at time intervals to a file and visualizing the results.
+ * See SingleWriterRecorder, HistogramLogWriter, and HisogramLogAnalyzer.
+ *
+ * TODO consider writing per interval histograms using HistogramLogWriter
+ */
+public class HdrHistogramProbe implements BenchmarkExecutionAwareProbe, BenchmarkTotalsOnlyProbe {
+
+  private final int lower;
+  private final long upper;
+  private final int numDigits;
+  private final Clock clock;
+  private long[] startTimes;
+  private Histogram[] histograms;
+
+  public HdrHistogramProbe() {
+    this(1, TimeUnit.HOURS.toNanos(5), 3, () -> System.nanoTime());
+  }
+
+  public HdrHistogramProbe(int lower, long upper, int numDigits, Clock clock) {
+    this.lower = lower;
+    this.upper = upper;
+    this.numDigits = numDigits;
+    this.clock = clock;
+  }
+
+  @Override
+  public void beforeExecute(int threadIdx) {
+      startTimes[threadIdx] = clock.currentTimeNanos();
+  }
+
+  @Override
+  public void afterExecute(int threadIdx) {
+      histograms[threadIdx].recordValue(clock.currentTimeNanos() - startTimes[threadIdx]);
+  }
+
+  @Override
+  public void start(BenchmarkDriver drv, BenchmarkConfiguration cfg) throws Exception {
+    int threads = cfg.threads();
+    start(threads);
+
+  }
+
+  void start(int threads) {
+    startTimes = new long[threads];
+    histograms = new Histogram[threads];
+
+    reset();
+  }
+
+  private void reset() {
+    for(int i =0; i < histograms.length; i++) {
+      histograms[i] = new Histogram(lower, upper, numDigits);
+    }
+  }
+
+  @Override
+  public void stop() {
+  }
+
+  @Override
+  public Collection<String> metaInfo() {
+    return Arrays.asList("Timestamp", "Mean latency, nanos", "99th percentile latency, nanos");
+  }
+
+  @Override
+  public Collection<BenchmarkProbePoint> points() {
+    Histogram aggregate = getHistogram();
+    reset();
+
+    long benchmarkEnd = clock.currentTimeNanos();
+    double percentile50 = aggregate.getMean();
+    long percentile99 = aggregate.getValueAtPercentile(99);
+
+    BenchmarkProbePoint point = new BenchmarkProbePoint(0, new double[] {percentile50, percentile99});
+
+    return Collections.singleton(point);
+  }
+
+  @Override
+  public void buildPoint(long time) {
+
+  }
+
+
+
+  public Histogram getHistogram() {
+    Histogram aggregate = new Histogram(lower, upper, numDigits);
+    for(Histogram histogram : histograms) {
+      aggregate.add(histogram);
+    }
+    return aggregate;
+  }
+}
diff --git a/harness/src/main/java/org/apache/geode/perftest/yardstick/YardstickTask.java b/harness/src/main/java/org/apache/geode/perftest/yardstick/YardstickTask.java
index 6c8cb7b..50ad949 100644
--- a/harness/src/main/java/org/apache/geode/perftest/yardstick/YardstickTask.java
+++ b/harness/src/main/java/org/apache/geode/perftest/yardstick/YardstickTask.java
@@ -90,7 +90,7 @@ public class YardstickTask implements Task {
     benchmark.setUp(cfg);
 
     TestDoneProbe testDoneProbe = new TestDoneProbe();
-    Collection<BenchmarkProbe> probes = Arrays.asList(new ThroughputLatencyProbe(),
+    Collection<BenchmarkProbe> probes = Arrays.asList(new HdrHistogramProbe(), new ThroughputLatencyProbe(),
         new PercentileProbe(), new DStatProbe(), new VmStatProbe(),
         testDoneProbe);
     BenchmarkLoader loader = new BenchmarkLoader();
diff --git a/harness/src/test/java/org/apache/geode/perftest/yardstick/HdrHistogramProbeTest.java b/harness/src/test/java/org/apache/geode/perftest/yardstick/HdrHistogramProbeTest.java
new file mode 100644
index 0000000..ba26933
--- /dev/null
+++ b/harness/src/test/java/org/apache/geode/perftest/yardstick/HdrHistogramProbeTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.
+ */
+
+package org.apache.geode.perftest.yardstick;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.HdrHistogram.Histogram;
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.Test;
+import org.yardstickframework.BenchmarkProbePoint;
+
+public class HdrHistogramProbeTest {
+
+  private HdrHistogramProbe probe;
+  private Clock clock;
+
+  @Before
+  public void setUp() {
+    clock = mock(Clock.class);
+    probe = new HdrHistogramProbe(1, 3_600_000, 3, clock);
+  }
+
+  @Test
+  public void recordsASingleValue() throws InterruptedException {
+    probe.start(8);
+    when(clock.currentTimeNanos()).thenReturn(0L);
+    probe.beforeExecute(1);
+    when(clock.currentTimeNanos()).thenReturn(2L);
+    probe.afterExecute(1);
+
+    Histogram histogram = probe.getHistogram();
+    assertEquals(2, histogram.getMaxValue());
+  }
+
+  @Test
+  public void recordsFromMultipleThreads() {
+    probe.start(3);
+
+    when(clock.currentTimeNanos()).thenReturn(0L);
+    probe.beforeExecute(0);
+
+    when(clock.currentTimeNanos()).thenReturn(1L);
+    probe.beforeExecute(2);
+
+    when(clock.currentTimeNanos()).thenReturn(1L);
+    probe.beforeExecute(1);
+
+    when(clock.currentTimeNanos()).thenReturn(3L);
+    probe.afterExecute(2);
+
+    when(clock.currentTimeNanos()).thenReturn(3L);
+    probe.afterExecute(1);
+
+    when(clock.currentTimeNanos()).thenReturn(4L);
+    probe.afterExecute(0);
+
+    assertEquals(4, probe.getHistogram().getMaxValue());
+    assertEquals(8.0/3.0, probe.getHistogram().getMean(), 0.01);
+  }
+
+  @Test
+  public void generatesASummaryBenchmarkPoint() throws Exception {
+    probe.start(1);
+    when(clock.currentTimeNanos()).thenReturn(0L);
+    probe.beforeExecute(0);
+    when(clock.currentTimeNanos()).thenReturn(2L);
+    probe.afterExecute(0);
+    probe.beforeExecute(0);
+    when(clock.currentTimeNanos()).thenReturn(6L);
+    probe.afterExecute(0);
+    when(clock.currentTimeNanos()).thenReturn(8L);
+
+    probe.stop();
+
+    Collection<BenchmarkProbePoint> points = probe.points();
+    assertEquals(1, points.size());
+    BenchmarkProbePoint point = points.iterator().next();
+
+    assertEquals(0, point.time());
+    Assertions.assertThat(point.values()).containsExactly(3.0, 4.0);
+  }
+}
\ No newline at end of file