Browse Source

HDFS-12678. Ozone: Corona: Add statistical information to json output. Contributed by Lokesh Jain.

Nanda kumar 7 years ago
parent
commit
34c2639777

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -236,6 +236,10 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <artifactId>curator-test</artifactId>
       <scope>test</scope>
     </dependency>
+      <dependency>
+          <groupId>com.codahale.metrics</groupId>
+          <artifactId>metrics-core</artifactId>
+      </dependency>
   </dependencies>
 
   <build>

+ 219 - 154
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/tools/Corona.java

@@ -17,6 +17,9 @@
 
 package org.apache.hadoop.ozone.tools;
 
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Snapshot;
+import com.codahale.metrics.UniformReservoir;
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.PropertyAccessor;
 import com.fasterxml.jackson.databind.ObjectMapper;
@@ -57,7 +60,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.Supplier;
 
-import static java.lang.Math.max;
 import static java.lang.Math.min;
 
 /**
@@ -93,6 +95,13 @@ import static java.lang.Math.min;
  */
 public final class Corona extends Configured implements Tool {
 
+  enum CoronaOps {
+    VOLUME_CREATE,
+    BUCKET_CREATE,
+    KEY_CREATE,
+    KEY_WRITE
+  }
+
   private static final String HELP = "help";
   private static final String MODE = "mode";
   private static final String SOURCE = "source";
@@ -116,6 +125,7 @@ public final class Corona extends Configured implements Tool {
   private static final String DURATION_FORMAT = "HH:mm:ss,SSS";
 
   private static final int KEY_SIZE_DEFAULT = 10240;
+  private static final int QUANTILES = 10;
 
   private static final Logger LOG =
       LoggerFactory.getLogger(Corona.class);
@@ -135,6 +145,7 @@ public final class Corona extends Configured implements Tool {
   private OzoneProtos.ReplicationType type;
   private OzoneProtos.ReplicationFactor factor;
 
+  private int threadPoolSize;
   private int keySize;
   private byte[] keyValue = null;
 
@@ -163,7 +174,7 @@ public final class Corona extends Configured implements Tool {
   private Long writeValidationFailureCount;
 
   private BlockingQueue<KeyValue> validationQueue;
-  private List<Double> threadThroughput;
+  private ArrayList<Histogram> histograms = new ArrayList<>();
 
   @VisibleForTesting
   Corona(Configuration conf) throws IOException {
@@ -180,6 +191,9 @@ public final class Corona extends Configured implements Tool {
     OzoneClientFactory.setConfiguration(conf);
     ozoneClient = OzoneClientFactory.getClient();
     objectStore = ozoneClient.getObjectStore();
+    for (CoronaOps ops : CoronaOps.values()) {
+      histograms.add(ops.ordinal(), new Histogram(new UniformReservoir()));
+    }
   }
 
   /**
@@ -201,13 +215,13 @@ public final class Corona extends Configured implements Tool {
       return 0;
     }
 
-    threadThroughput = Collections.synchronizedList(new ArrayList<Double>());
-
     keyValue =
         DFSUtil.string2Bytes(RandomStringUtils.randomAscii(keySize - 36));
 
     LOG.info("Number of Threads: " + numOfThreads);
-    processor = Executors.newFixedThreadPool(Integer.parseInt(numOfThreads));
+    threadPoolSize =
+        min(Integer.parseInt(numOfVolumes), Integer.parseInt(numOfThreads));
+    processor = Executors.newFixedThreadPool(threadPoolSize);
     addShutdownHook();
     if (mode.equals("online")) {
       LOG.info("Mode: online");
@@ -439,42 +453,29 @@ public final class Corona extends Configured implements Tool {
    * @param out PrintStream
    */
   private void printStats(PrintStream out) {
-    int threadCount = Integer.parseInt(numOfThreads);
-
     long endTime = System.nanoTime() - startTime;
     String execTime = DurationFormatUtils
         .formatDuration(TimeUnit.NANOSECONDS.toMillis(endTime),
             DURATION_FORMAT);
-    String prettyTotalVolumeTime = DurationFormatUtils
-        .formatDuration(TimeUnit.NANOSECONDS.toMillis(volumeCreationTime.get()),
-            DURATION_FORMAT);
-    String prettyTotalBucketTime = DurationFormatUtils
-        .formatDuration(TimeUnit.NANOSECONDS.toMillis(bucketCreationTime.get()),
-            DURATION_FORMAT);
-    String prettyTotalKeyCreationTime = DurationFormatUtils
-        .formatDuration(TimeUnit.NANOSECONDS.toMillis(keyCreationTime.get()),
-            DURATION_FORMAT);
-    String prettyTotalKeyWriteTime = DurationFormatUtils
-        .formatDuration(TimeUnit.NANOSECONDS.toMillis(keyWriteTime.get()),
-            DURATION_FORMAT);
 
-    long volumeTime =
-        TimeUnit.NANOSECONDS.toMillis(volumeCreationTime.get()) / threadCount;
+    long volumeTime = TimeUnit.NANOSECONDS.toMillis(volumeCreationTime.get())
+        / threadPoolSize;
     String prettyAverageVolumeTime =
         DurationFormatUtils.formatDuration(volumeTime, DURATION_FORMAT);
 
-    long bucketTime =
-        TimeUnit.NANOSECONDS.toMillis(bucketCreationTime.get()) / threadCount;
+    long bucketTime = TimeUnit.NANOSECONDS.toMillis(bucketCreationTime.get())
+        / threadPoolSize;
     String prettyAverageBucketTime =
         DurationFormatUtils.formatDuration(bucketTime, DURATION_FORMAT);
 
     long averageKeyCreationTime =
-        TimeUnit.NANOSECONDS.toMillis(keyCreationTime.get()) / threadCount;
+        TimeUnit.NANOSECONDS.toMillis(keyCreationTime.get())
+            / threadPoolSize;
     String prettyAverageKeyCreationTime = DurationFormatUtils
         .formatDuration(averageKeyCreationTime, DURATION_FORMAT);
 
     long averageKeyWriteTime =
-        TimeUnit.NANOSECONDS.toMillis(keyWriteTime.get()) / threadCount;
+        TimeUnit.NANOSECONDS.toMillis(keyWriteTime.get()) / threadPoolSize;
     String prettyAverageKeyWriteTime = DurationFormatUtils
         .formatDuration(averageKeyWriteTime, DURATION_FORMAT);
 
@@ -487,10 +488,6 @@ public final class Corona extends Configured implements Tool {
     out.println("Number of Keys added: " + numberOfKeysAdded);
     out.println("Ratis replication factor: " + factor.name());
     out.println("Ratis replication type: " + type.name());
-    out.println("Time spent in volume creation: " + prettyTotalVolumeTime);
-    out.println("Time spent in bucket creation: " + prettyTotalBucketTime);
-    out.println("Time spent in key creation: " + prettyTotalKeyCreationTime);
-    out.println("Time spent in key write: " + prettyTotalKeyWriteTime);
     out.println(
         "Average Time spent in volume creation: " + prettyAverageVolumeTime);
     out.println(
@@ -515,17 +512,47 @@ public final class Corona extends Configured implements Tool {
     out.println("***************************************************");
 
     if (jsonDir != null) {
-      CoronaJobInfo jobInfo = new CoronaJobInfo()
-          .setExecTime(execTime)
+
+      String[][] quantileTime =
+          new String[CoronaOps.values().length][QUANTILES + 1];
+      String[] deviations = new String[CoronaOps.values().length];
+      String[] means = new String[CoronaOps.values().length];
+      for (CoronaOps ops : CoronaOps.values()) {
+        Snapshot snapshot = histograms.get(ops.ordinal()).getSnapshot();
+        for (int i = 0; i <= QUANTILES; i++) {
+          quantileTime[ops.ordinal()][i] = DurationFormatUtils.formatDuration(
+              TimeUnit.NANOSECONDS
+                  .toMillis((long) snapshot.getValue((1.0 / QUANTILES) * i)),
+              DURATION_FORMAT);
+        }
+        deviations[ops.ordinal()] = DurationFormatUtils.formatDuration(
+            TimeUnit.NANOSECONDS.toMillis((long) snapshot.getStdDev()),
+            DURATION_FORMAT);
+        means[ops.ordinal()] = DurationFormatUtils.formatDuration(
+            TimeUnit.NANOSECONDS.toMillis((long) snapshot.getMean()),
+            DURATION_FORMAT);
+      }
+
+      CoronaJobInfo jobInfo = new CoronaJobInfo().setExecTime(execTime)
           .setGitBaseRevision(VersionInfo.getRevision())
-          .setAverageVolumeCreationTime(prettyAverageVolumeTime)
-          .setAverageBucketCreationTime(prettyAverageBucketTime)
-          .setAverageKeyCreationTime(prettyAverageKeyCreationTime)
-          .setAverageKeyWriteTime(prettyAverageKeyWriteTime)
-          .setTotalVolumeCreationTime(prettyTotalVolumeTime)
-          .setTotalBucketCreationTime(prettyTotalBucketTime)
-          .setTotalKeyCreationTime(prettyTotalKeyCreationTime)
-          .setTotalKeyWriteTime(prettyTotalKeyWriteTime);
+          .setMeanVolumeCreateTime(means[CoronaOps.VOLUME_CREATE.ordinal()])
+          .setDeviationVolumeCreateTime(
+              deviations[CoronaOps.VOLUME_CREATE.ordinal()])
+          .setTenQuantileVolumeCreateTime(
+              quantileTime[CoronaOps.VOLUME_CREATE.ordinal()])
+          .setMeanBucketCreateTime(means[CoronaOps.BUCKET_CREATE.ordinal()])
+          .setDeviationBucketCreateTime(
+              deviations[CoronaOps.BUCKET_CREATE.ordinal()])
+          .setTenQuantileBucketCreateTime(
+              quantileTime[CoronaOps.BUCKET_CREATE.ordinal()])
+          .setMeanKeyCreateTime(means[CoronaOps.KEY_CREATE.ordinal()])
+          .setDeviationKeyCreateTime(deviations[CoronaOps.KEY_CREATE.ordinal()])
+          .setTenQuantileKeyCreateTime(
+              quantileTime[CoronaOps.KEY_CREATE.ordinal()])
+          .setMeanKeyWriteTime(means[CoronaOps.KEY_WRITE.ordinal()])
+          .setDeviationKeyWriteTime(deviations[CoronaOps.KEY_WRITE.ordinal()])
+          .setTenQuantileKeyWriteTime(
+              quantileTime[CoronaOps.KEY_WRITE.ordinal()]);
       String jsonName =
           new SimpleDateFormat("yyyyMMddHHmmss").format(Time.now()) + ".json";
       String jsonPath = jsonDir + "/" + jsonName;
@@ -662,21 +689,33 @@ public final class Corona extends Configured implements Tool {
 
     private int totalBuckets;
     private int totalKeys;
-    private OzoneVolume volume;
+    private String volumeName;
 
-    OfflineProcessor(String volumeName) throws Exception {
+    OfflineProcessor(String volumeName) {
       this.totalBuckets = Integer.parseInt(numOfBuckets);
       this.totalKeys = Integer.parseInt(numOfKeys);
-      LOG.trace("Creating volume: {}", volumeName);
-      long start = System.nanoTime();
-      objectStore.createVolume(volumeName);
-      volumeCreationTime.getAndAdd(System.nanoTime() - start);
-      numberOfVolumesCreated.getAndIncrement();
-      volume = objectStore.getVolume(volumeName);
+      this.volumeName = volumeName;
     }
 
     @Override
     public void run() {
+      LOG.trace("Creating volume: {}", volumeName);
+      long start = System.nanoTime();
+      OzoneVolume volume;
+      try {
+        objectStore.createVolume(volumeName);
+        long volumeCreationDuration = System.nanoTime() - start;
+        volumeCreationTime.getAndAdd(volumeCreationDuration);
+        histograms.get(CoronaOps.VOLUME_CREATE.ordinal())
+            .update(volumeCreationDuration);
+        numberOfVolumesCreated.getAndIncrement();
+        volume = objectStore.getVolume(volumeName);
+      } catch (IOException e) {
+        exception = true;
+        LOG.error("Could not create volume", e);
+        return;
+      }
+
       Long threadKeyWriteTime = 0L;
       for (int j = 0; j < totalBuckets; j++) {
         String bucketName = "bucket-" + j + "-" +
@@ -684,9 +723,12 @@ public final class Corona extends Configured implements Tool {
         try {
           LOG.trace("Creating bucket: {} in volume: {}",
               bucketName, volume.getName());
-          long start = System.nanoTime();
+          start = System.nanoTime();
           volume.createBucket(bucketName);
-          bucketCreationTime.getAndAdd(System.nanoTime() - start);
+          long bucketCreationDuration = System.nanoTime() - start;
+          histograms.get(CoronaOps.BUCKET_CREATE.ordinal())
+              .update(bucketCreationDuration);
+          bucketCreationTime.getAndAdd(bucketCreationDuration);
           numberOfBucketsCreated.getAndIncrement();
           OzoneBucket bucket = volume.getBucket(bucketName);
           for (int k = 0; k < totalKeys; k++) {
@@ -700,12 +742,18 @@ public final class Corona extends Configured implements Tool {
               long keyCreateStart = System.nanoTime();
               OzoneOutputStream os =
                   bucket.createKey(key, keySize, type, factor);
-              keyCreationTime.getAndAdd(System.nanoTime() - keyCreateStart);
+              long keyCreationDuration = System.nanoTime() - keyCreateStart;
+              histograms.get(CoronaOps.KEY_CREATE.ordinal())
+                  .update(keyCreationDuration);
+              keyCreationTime.getAndAdd(keyCreationDuration);
               long keyWriteStart = System.nanoTime();
               os.write(keyValue);
               os.write(randomValue);
               os.close();
-              threadKeyWriteTime += System.nanoTime() - keyWriteStart;
+              long keyWriteDuration = System.nanoTime() - keyWriteStart;
+              threadKeyWriteTime += keyWriteDuration;
+              histograms.get(CoronaOps.KEY_WRITE.ordinal())
+                  .update(keyWriteDuration);
               totalBytesWritten.getAndAdd(keySize);
               numberOfKeysAdded.getAndIncrement();
               if (validateWrites) {
@@ -730,13 +778,6 @@ public final class Corona extends Configured implements Tool {
       }
 
       keyWriteTime.getAndAdd(threadKeyWriteTime);
-      boolean success = threadThroughput.add(
-          (totalBuckets * totalKeys * keySize * 1.0) / TimeUnit.NANOSECONDS
-              .toSeconds(threadKeyWriteTime));
-      if (!success) {
-        LOG.warn("Throughput could not be added for thread id: {}",
-            Thread.currentThread().getId());
-      }
     }
 
   }
@@ -751,14 +792,6 @@ public final class Corona extends Configured implements Tool {
     private String numOfKeys;
     private String numOfThreads;
     private String mode;
-    private String totalBucketCreationTime;
-    private String totalVolumeCreationTime;
-    private String totalKeyCreationTime;
-    private String totalKeyWriteTime;
-    private String averageBucketCreationTime;
-    private String averageVolumeCreationTime;
-    private String averageKeyCreationTime;
-    private String averageKeyWriteTime;
     private String dataWritten;
     private String execTime;
     private String replicationFactor;
@@ -766,11 +799,24 @@ public final class Corona extends Configured implements Tool {
 
     private int keySize;
 
-    private String[] threadThroughputPerSecond;
-    private String minThreadThroughputPerSecond;
-    private String maxThreadThroughputPerSecond;
     private String totalThroughputPerSecond;
 
+    private String meanVolumeCreateTime;
+    private String deviationVolumeCreateTime;
+    private String[] tenQuantileVolumeCreateTime;
+
+    private String meanBucketCreateTime;
+    private String deviationBucketCreateTime;
+    private String[] tenQuantileBucketCreateTime;
+
+    private String meanKeyCreateTime;
+    private String deviationKeyCreateTime;
+    private String[] tenQuantileKeyCreateTime;
+
+    private String meanKeyWriteTime;
+    private String deviationKeyWriteTime;
+    private String[] tenQuantileKeyWriteTime;
+
     private CoronaJobInfo() {
       this.status = exception ? "Failed" : "Success";
       this.numOfVolumes = Corona.this.numOfVolumes;
@@ -787,37 +833,26 @@ public final class Corona extends Configured implements Tool {
           Long.parseLong(numOfVolumes) * Long.parseLong(numOfBuckets) * Long
               .parseLong(numOfKeys) * keySize;
       this.dataWritten = getInStorageUnits((double) totalBytes);
-
-      threadThroughputPerSecond = new String[Integer.parseInt(numOfThreads)];
-      double minThreadThroughput = Double.MAX_VALUE, maxThreadThroughput = 0.0,
-          totalThroughput = 0.0;
-      int i = 0;
-      for (Double throughput : Corona.this.threadThroughput) {
-        minThreadThroughput = min(throughput, minThreadThroughput);
-        maxThreadThroughput = max(throughput, maxThreadThroughput);
-        totalThroughput += throughput;
-        threadThroughputPerSecond[i++] = getInStorageUnits(throughput);
-      }
-      minThreadThroughputPerSecond = getInStorageUnits(minThreadThroughput);
-      maxThreadThroughputPerSecond = getInStorageUnits(maxThreadThroughput);
-      totalThroughputPerSecond = getInStorageUnits(totalThroughput);
+      this.totalThroughputPerSecond = getInStorageUnits(
+          (totalBytes * 1.0) / TimeUnit.NANOSECONDS
+              .toSeconds(Corona.this.keyWriteTime.get() / threadPoolSize));
     }
 
     private String getInStorageUnits(Double value) {
       double size;
       OzoneQuota.Units unit;
-      if ((long) (value / OzoneConsts.KB) == 0) {
-        size = value / OzoneConsts.KB;
-        unit = OzoneQuota.Units.KB;
-      } else if ((long) (value / OzoneConsts.MB) == 0) {
-        size = value / OzoneConsts.MB;
-        unit = OzoneQuota.Units.MB;
-      } else if ((long) (value / OzoneConsts.GB) == 0) {
-        size = value / OzoneConsts.GB;
-        unit = OzoneQuota.Units.GB;
-      } else if ((long) (value / OzoneConsts.TB) == 0) {
+      if ((long) (value / OzoneConsts.TB) != 0) {
         size = value / OzoneConsts.TB;
         unit = OzoneQuota.Units.TB;
+      } else if ((long) (value / OzoneConsts.GB) != 0) {
+        size = value / OzoneConsts.GB;
+        unit = OzoneQuota.Units.GB;
+      } else if ((long) (value / OzoneConsts.MB) != 0) {
+        size = value / OzoneConsts.MB;
+        unit = OzoneQuota.Units.MB;
+      } else if ((long) (value / OzoneConsts.KB) != 0) {
+        size = value / OzoneConsts.KB;
+        unit = OzoneQuota.Units.KB;
       } else {
         size = value;
         unit = OzoneQuota.Units.BYTES;
@@ -830,55 +865,78 @@ public final class Corona extends Configured implements Tool {
       return this;
     }
 
-    public CoronaJobInfo setTotalBucketCreationTime(
-        String totalBucketCreationTimeVal) {
-      totalBucketCreationTime = totalBucketCreationTimeVal;
+    public CoronaJobInfo setExecTime(String execTimeVal) {
+      execTime = execTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setTotalVolumeCreationTime(
-        String totalVolumeCreationTimeVal) {
-      totalVolumeCreationTime = totalVolumeCreationTimeVal;
+    public CoronaJobInfo setMeanKeyWriteTime(String deviationKeyWriteTimeVal) {
+      this.meanKeyWriteTime = deviationKeyWriteTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setTotalKeyCreationTime(
-        String totalKeyCreationTimeVal) {
-      totalKeyCreationTime = totalKeyCreationTimeVal;
+    public CoronaJobInfo setDeviationKeyWriteTime(
+        String deviationKeyWriteTimeVal) {
+      this.deviationKeyWriteTime = deviationKeyWriteTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setTotalKeyWriteTime(String totalKeyWriteTimeVal) {
-      totalKeyWriteTime = totalKeyWriteTimeVal;
+    public CoronaJobInfo setTenQuantileKeyWriteTime(
+        String[] tenQuantileKeyWriteTimeVal) {
+      this.tenQuantileKeyWriteTime = tenQuantileKeyWriteTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setAverageBucketCreationTime(
-        String averageBucketCreationTimeVal) {
-      averageBucketCreationTime = averageBucketCreationTimeVal;
+    public CoronaJobInfo setMeanKeyCreateTime(String deviationKeyWriteTimeVal) {
+      this.meanKeyCreateTime = deviationKeyWriteTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setAverageVolumeCreationTime(
-        String averageVolumeCreationTimeVal) {
-      averageVolumeCreationTime = averageVolumeCreationTimeVal;
+    public CoronaJobInfo setDeviationKeyCreateTime(
+        String deviationKeyCreateTimeVal) {
+      this.deviationKeyCreateTime = deviationKeyCreateTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setAverageKeyCreationTime(
-        String averageKeyCreationTimeVal) {
-      averageKeyCreationTime = averageKeyCreationTimeVal;
+    public CoronaJobInfo setTenQuantileKeyCreateTime(
+        String[] tenQuantileKeyCreateTimeVal) {
+      this.tenQuantileKeyCreateTime = tenQuantileKeyCreateTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setAverageKeyWriteTime(
-        String averageKeyWriteTimeVal) {
-      averageKeyWriteTime = averageKeyWriteTimeVal;
+    public CoronaJobInfo setMeanBucketCreateTime(
+        String deviationKeyWriteTimeVal) {
+      this.meanBucketCreateTime = deviationKeyWriteTimeVal;
       return this;
     }
 
-    public CoronaJobInfo setExecTime(String execTimeVal) {
-      execTime = execTimeVal;
+    public CoronaJobInfo setDeviationBucketCreateTime(
+        String deviationBucketCreateTimeVal) {
+      this.deviationBucketCreateTime = deviationBucketCreateTimeVal;
+      return this;
+    }
+
+    public CoronaJobInfo setTenQuantileBucketCreateTime(
+        String[] tenQuantileBucketCreateTimeVal) {
+      this.tenQuantileBucketCreateTime = tenQuantileBucketCreateTimeVal;
+      return this;
+    }
+
+    public CoronaJobInfo setMeanVolumeCreateTime(
+        String deviationKeyWriteTimeVal) {
+      this.meanVolumeCreateTime = deviationKeyWriteTimeVal;
+      return this;
+    }
+
+    public CoronaJobInfo setDeviationVolumeCreateTime(
+        String deviationVolumeCreateTimeVal) {
+      this.deviationVolumeCreateTime = deviationVolumeCreateTimeVal;
+      return this;
+    }
+
+    public CoronaJobInfo setTenQuantileVolumeCreateTime(
+        String[] tenQuantileVolumeCreateTimeVal) {
+      this.tenQuantileVolumeCreateTime = tenQuantileVolumeCreateTimeVal;
       return this;
     }
 
@@ -906,78 +964,85 @@ public final class Corona extends Configured implements Tool {
       return mode;
     }
 
-    public String getTotalBucketCreationTime() {
-      return totalBucketCreationTime;
+    public String getExecTime() {
+      return execTime;
     }
 
-    public String getTotalVolumeCreationTime() {
-      return totalVolumeCreationTime;
+    public String getReplicationFactor() {
+      return replicationFactor;
     }
 
-    public String getTotalKeyCreationTime() {
-      return totalKeyCreationTime;
+    public String getReplicationType() {
+      return replicationType;
     }
 
-    public String getAverageBucketCreationTime() {
-      return averageBucketCreationTime;
+    public String getStatus() {
+      return status;
     }
 
-    public String getTotalKeyWriteTime() {
-      return totalKeyWriteTime;
+    public int getKeySize() {
+      return keySize;
     }
 
-    public String getAverageKeyWriteTime() {
-      return averageKeyWriteTime;
+    public String getGitBaseRevision() {
+      return gitBaseRevision;
     }
 
-    public String getAverageVolumeCreationTime() {
-      return averageVolumeCreationTime;
+    public String getDataWritten() {
+      return dataWritten;
     }
 
-    public String getAverageKeyCreationTime() {
-      return averageKeyCreationTime;
+    public String getTotalThroughputPerSecond() {
+      return totalThroughputPerSecond;
     }
 
-    public String getExecTime() {
-      return execTime;
+    public String getMeanVolumeCreateTime() {
+      return meanVolumeCreateTime;
     }
 
-    public String getReplicationFactor() {
-      return replicationFactor;
+    public String getDeviationVolumeCreateTime() {
+      return deviationVolumeCreateTime;
     }
 
-    public String getReplicationType() {
-      return replicationType;
+    public String[] getTenQuantileVolumeCreateTime() {
+      return tenQuantileVolumeCreateTime;
     }
 
-    public String getStatus() {
-      return status;
+    public String getMeanBucketCreateTime() {
+      return meanBucketCreateTime;
     }
 
-    public int getKeySize() {
-      return keySize;
+    public String getDeviationBucketCreateTime() {
+      return deviationBucketCreateTime;
     }
 
-    public String getGitBaseRevision() {
-      return gitBaseRevision;
+    public String[] getTenQuantileBucketCreateTime() {
+      return tenQuantileBucketCreateTime;
     }
 
-    public String getMinThreadThroughputPerSecond() {
-      return minThreadThroughputPerSecond;
+    public String getMeanKeyCreateTime() {
+      return meanKeyCreateTime;
     }
 
-    public String getMaxThreadThroughputPerSecond() {
-      return maxThreadThroughputPerSecond;
+    public String getDeviationKeyCreateTime() {
+      return deviationKeyCreateTime;
     }
 
-    public String getDataWritten() {
-      return dataWritten;
+    public String[] getTenQuantileKeyCreateTime() {
+      return tenQuantileKeyCreateTime;
     }
 
-    public String getTotalThroughput() {
-      return totalThroughputPerSecond;
+    public String getMeanKeyWriteTime() {
+      return meanKeyWriteTime;
     }
 
+    public String getDeviationKeyWriteTime() {
+      return deviationKeyWriteTime;
+    }
+
+    public String[] getTenQuantileKeyWriteTime() {
+      return tenQuantileKeyWriteTime;
+    }
   }
 
   private class ProgressBar implements Runnable {