You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nk...@apache.org on 2013/11/22 22:16:11 UTC

svn commit: r1544683 - /hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java

Author: nkeywal
Date: Fri Nov 22 21:16:11 2013
New Revision: 1544683

URL: http://svn.apache.org/r1544683
Log:
HBASE-10007 PerformanceEvaluation: Add sampling and latency collection to randomRead test (Nick Dimiduk)

Modified:
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java

Modified: hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java?rev=1544683&r1=1544682&r2=1544683&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java (original)
+++ hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java Fri Nov 22 21:16:11 2013
@@ -22,7 +22,9 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.PrintStream;
-import java.io.File;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.text.DecimalFormat;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Date;
@@ -83,7 +85,6 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.LineReader;
 
-
 /**
  * Script used evaluating HBase performance and scalability.  Runs a HBase
  * client that steps through one of a set of hardcoded tests or 'experiments'
@@ -103,46 +104,49 @@ import org.apache.hadoop.util.LineReader
 public class PerformanceEvaluation extends Configured implements Tool {
   protected static final Log LOG = LogFactory.getLog(PerformanceEvaluation.class.getName());
 
-  private static final int DEFAULT_ROW_PREFIX_LENGTH = 16;
-  private static final int VALUE_LENGTH = 1000;
-  private static final int ONE_GB = 1024 * 1024 * 1000;
-  private static final int ROWS_PER_GB = ONE_GB / VALUE_LENGTH;
-
-  public static final byte[] COMPRESSION = Bytes.toBytes("NONE");
-  public static final TableName TABLE_NAME =
-      TableName.valueOf("TestTable");
+  public static final TableName TABLE_NAME = TableName.valueOf("TestTable");
   public static final byte[] FAMILY_NAME = Bytes.toBytes("info");
   public static final byte[] QUALIFIER_NAME = Bytes.toBytes("data");
+  public static final int VALUE_LENGTH = 1000;
+  public static final int ROW_LENGTH = 26;
 
-  protected HTableDescriptor TABLE_DESCRIPTOR;
+  private static final int ONE_GB = 1024 * 1024 * 1000;
+  private static final int ROWS_PER_GB = ONE_GB / VALUE_LENGTH;
+  private static final DecimalFormat FMT = new DecimalFormat("0.##");
+  private static final MathContext CXT = MathContext.DECIMAL64;
+  private static final BigDecimal MS_PER_SEC = BigDecimal.valueOf(1000);
+  private static final BigDecimal BYTES_PER_MB = BigDecimal.valueOf(1024 * 1024);
 
+  protected HTableDescriptor TABLE_DESCRIPTOR;
   protected Map<String, CmdDescriptor> commands = new TreeMap<String, CmdDescriptor>();
 
   private boolean nomapred = false;
-  private int rowPrefixLength = DEFAULT_ROW_PREFIX_LENGTH;
   private int N = 1;
   private int R = ROWS_PER_GB;
+  private float sampleRate = 1.0f;
   private TableName tableName = TABLE_NAME;
   private Compression.Algorithm compression = Compression.Algorithm.NONE;
   private DataBlockEncoding blockEncoding = DataBlockEncoding.NONE;
   private boolean flushCommits = true;
   private boolean writeToWAL = true;
   private boolean inMemoryCF = false;
+  private boolean reportLatency = false;
   private int presplitRegions = 0;
   private HConnection connection;
 
   private static final Path PERF_EVAL_DIR = new Path("performance_evaluation");
-  /**
-   * Regex to parse lines in input file passed to mapreduce task.
-   */
+
+  /** Regex to parse lines in input file passed to mapreduce task. */
   public static final Pattern LINE_PATTERN =
     Pattern.compile("tableName=(\\w+),\\s+" +
         "startRow=(\\d+),\\s+" +
         "perClientRunRows=(\\d+),\\s+" +
         "totalRows=(\\d+),\\s+" +
+        "sampleRate=([-+]?[0-9]*\\.?[0-9]+),\\s+" +
         "clients=(\\d+),\\s+" +
         "flushCommits=(\\w+),\\s+" +
-        "writeToWAL=(\\w+)");
+        "writeToWAL=(\\w+),\\s+" +
+        "reportLatency=(\\w+)");
 
   /**
    * Enum for map metrics.  Keep it out here rather than inside in the Map
@@ -152,8 +156,8 @@ public class PerformanceEvaluation exten
     /** elapsed time */
     ELAPSED_TIME,
     /** number of rows */
-    ROWS}
-
+    ROWS
+  }
 
   /**
    * Constructor
@@ -216,28 +220,26 @@ public class PerformanceEvaluation exten
     private int startRow = 0;
     private int rows = 0;
     private int totalRows = 0;
+    private float sampleRate = 1.0f;
     private int clients = 0;
     private boolean flushCommits = false;
     private boolean writeToWAL = true;
+    private boolean reportLatency = false;
 
-    public PeInputSplit() {
-      this.startRow = 0;
-      this.rows = 0;
-      this.totalRows = 0;
-      this.clients = 0;
-      this.flushCommits = false;
-      this.writeToWAL = true;
-    }
+    public PeInputSplit() {}
 
-    public PeInputSplit(TableName tableName, int startRow, int rows, int totalRows, int clients,
-        boolean flushCommits, boolean writeToWAL) {
+    public PeInputSplit(TableName tableName, int startRow, int rows, int totalRows,
+        float sampleRate, int clients, boolean flushCommits, boolean writeToWAL,
+        boolean reportLatency) {
       this.tableName = tableName;
       this.startRow = startRow;
       this.rows = rows;
       this.totalRows = totalRows;
+      this.sampleRate = sampleRate;
       this.clients = clients;
       this.flushCommits = flushCommits;
       this.writeToWAL = writeToWAL;
+      this.reportLatency = reportLatency;
     }
 
     @Override
@@ -250,9 +252,11 @@ public class PerformanceEvaluation exten
       this.startRow = in.readInt();
       this.rows = in.readInt();
       this.totalRows = in.readInt();
+      this.sampleRate = in.readFloat();
       this.clients = in.readInt();
       this.flushCommits = in.readBoolean();
       this.writeToWAL = in.readBoolean();
+      this.reportLatency = in.readBoolean();
     }
 
     @Override
@@ -263,9 +267,11 @@ public class PerformanceEvaluation exten
       out.writeInt(startRow);
       out.writeInt(rows);
       out.writeInt(totalRows);
+      out.writeFloat(sampleRate);
       out.writeInt(clients);
       out.writeBoolean(flushCommits);
       out.writeBoolean(writeToWAL);
+      out.writeBoolean(reportLatency);
     }
 
     @Override
@@ -294,6 +300,10 @@ public class PerformanceEvaluation exten
       return totalRows;
     }
 
+    public float getSampleRate() {
+      return sampleRate;
+    }
+
     public int getClients() {
       return clients;
     }
@@ -305,6 +315,10 @@ public class PerformanceEvaluation exten
     public boolean isWriteToWAL() {
       return writeToWAL;
     }
+
+    public boolean isReportLatency() {
+      return reportLatency;
+    }
   }
 
   /**
@@ -339,22 +353,26 @@ public class PerformanceEvaluation exten
             int startRow = Integer.parseInt(m.group(2));
             int rows = Integer.parseInt(m.group(3));
             int totalRows = Integer.parseInt(m.group(4));
-            int clients = Integer.parseInt(m.group(5));
-            boolean flushCommits = Boolean.parseBoolean(m.group(6));
-            boolean writeToWAL = Boolean.parseBoolean(m.group(7));
+            float sampleRate = Float.parseFloat(m.group(5));
+            int clients = Integer.parseInt(m.group(6));
+            boolean flushCommits = Boolean.parseBoolean(m.group(7));
+            boolean writeToWAL = Boolean.parseBoolean(m.group(8));
+            boolean reportLatency = Boolean.parseBoolean(m.group(9));
 
             LOG.debug("tableName=" + tableName +
                       " split["+ splitList.size() + "] " +
                       " startRow=" + startRow +
                       " rows=" + rows +
                       " totalRows=" + totalRows +
+                      " sampleRate=" + sampleRate +
                       " clients=" + clients +
                       " flushCommits=" + flushCommits +
-                      " writeToWAL=" + writeToWAL);
+                      " writeToWAL=" + writeToWAL +
+                      " reportLatency=" + reportLatency);
 
             PeInputSplit newSplit =
-              new PeInputSplit(tableName, startRow, rows, totalRows, clients,
-                flushCommits, writeToWAL);
+              new PeInputSplit(tableName, startRow, rows, totalRows, sampleRate, clients,
+                flushCommits, writeToWAL, reportLatency);
             splitList.add(newSplit);
           }
         }
@@ -391,7 +409,7 @@ public class PerformanceEvaluation exten
         }
 
         key = NullWritable.get();
-        value = (PeInputSplit)split;
+        value = split;
 
         readOver = true;
         return true;
@@ -475,9 +493,9 @@ public class PerformanceEvaluation exten
       // Evaluation task
       pe.tableName = value.getTableName();
       long elapsedTime = this.pe.runOneClient(this.cmd, value.getStartRow(),
-                                  value.getRows(), value.getTotalRows(),
-                                  value.isFlushCommits(), value.isWriteToWAL(),
-                                  HConnectionManager.createConnection(context.getConfiguration()), status);
+          value.getRows(), value.getTotalRows(), value.getSampleRate(),
+          value.isFlushCommits(), value.isWriteToWAL(), value.isReportLatency(),
+          HConnectionManager.createConnection(context.getConfiguration()), status);
       // Collect how much time the thing took. Report as map output and
       // to the ELAPSED_TIME counter.
       context.getCounter(Counter.ELAPSED_TIME).increment(elapsedTime);
@@ -516,8 +534,7 @@ public class PerformanceEvaluation exten
         LOG.info("Table " + tableDescriptor + " created");
       }
     }
-    boolean tableExists = admin.tableExists(tableDescriptor.getTableName());
-    return tableExists;
+    return admin.tableExists(tableDescriptor.getTableName());
   }
 
   protected HTableDescriptor getTableDescriptor() {
@@ -578,11 +595,13 @@ public class PerformanceEvaluation exten
     final List<Thread> threads = new ArrayList<Thread>(this.N);
     final long[] timings = new long[this.N];
     final int perClientRows = R/N;
+    final float sampleRate = this.sampleRate;
     final TableName tableName = this.tableName;
     final DataBlockEncoding encoding = this.blockEncoding;
     final boolean flushCommits = this.flushCommits;
     final Compression.Algorithm compression = this.compression;
     final boolean writeToWal = this.writeToWAL;
+    final boolean reportLatency = this.reportLatency;
     final int preSplitRegions = this.presplitRegions;
     final HConnection connection = HConnectionManager.createConnection(getConf());
     for (int i = 0; i < this.N; i++) {
@@ -599,11 +618,13 @@ public class PerformanceEvaluation exten
           pe.writeToWAL = writeToWal;
           pe.presplitRegions = preSplitRegions;
           pe.N = N;
+          pe.sampleRate = sampleRate;
+          pe.reportLatency = reportLatency;
           pe.connection = connection;
           try {
             long elapsedTime = pe.runOneClient(cmd, index * perClientRows,
-               perClientRows, R,
-                flushCommits, writeToWAL, connection, new Status() {
+               perClientRows, R, sampleRate,
+                flushCommits, writeToWAL, reportLatency, connection, new Status() {
                   public void setStatus(final String msg) throws IOException {
                     LOG.info("client-" + getName() + " " + msg);
                   }
@@ -712,9 +733,11 @@ public class PerformanceEvaluation exten
           ", startRow=" + ((j * perClientRows) + (i * (perClientRows/10))) +
           ", perClientRunRows=" + (perClientRows / 10) +
           ", totalRows=" + this.R +
+          ", sampleRate=" + this.sampleRate +
           ", clients=" + this.N +
           ", flushCommits=" + this.flushCommits +
-          ", writeToWAL=" + this.writeToWAL;
+          ", writeToWAL=" + this.writeToWAL +
+          ", reportLatency=" + this.reportLatency;
           int hash = h.hash(Bytes.toBytes(s));
           m.put(hash, s);
         }
@@ -763,25 +786,26 @@ public class PerformanceEvaluation exten
     private int startRow;
     private int perClientRunRows;
     private int totalRows;
+    private float sampleRate;
     private int numClientThreads;
     private TableName tableName;
     private boolean flushCommits;
     private boolean writeToWAL = true;
+    private boolean reportLatency;
     private HConnection connection;
 
-    TestOptions() {
-    }
-
-    TestOptions(int startRow, int perClientRunRows, int totalRows,
-                int numClientThreads, TableName tableName,
-                boolean flushCommits, boolean writeToWAL, HConnection connection) {
+    TestOptions(int startRow, int perClientRunRows, int totalRows, float sampleRate,
+        int numClientThreads, TableName tableName, boolean flushCommits, boolean writeToWAL,
+        boolean reportLatency, HConnection connection) {
       this.startRow = startRow;
       this.perClientRunRows = perClientRunRows;
       this.totalRows = totalRows;
+      this.sampleRate = sampleRate;
       this.numClientThreads = numClientThreads;
       this.tableName = tableName;
       this.flushCommits = flushCommits;
       this.writeToWAL = writeToWAL;
+      this.reportLatency = reportLatency;
       this.connection = connection;
     }
 
@@ -797,6 +821,10 @@ public class PerformanceEvaluation exten
       return totalRows;
     }
 
+    public float getSampleRate() {
+      return sampleRate;
+    }
+
     public int getNumClientThreads() {
       return numClientThreads;
     }
@@ -813,6 +841,10 @@ public class PerformanceEvaluation exten
       return writeToWAL;
     }
 
+    public boolean isReportLatency() {
+      return reportLatency;
+    }
+
     public HConnection getConnection() {
       return connection;
     }
@@ -835,12 +867,14 @@ public class PerformanceEvaluation exten
     protected final int startRow;
     protected final int perClientRunRows;
     protected final int totalRows;
+    protected final float sampleRate;
     private final Status status;
     protected TableName tableName;
     protected HTableInterface table;
     protected volatile Configuration conf;
     protected boolean flushCommits;
     protected boolean writeToWAL;
+    protected boolean reportLatency;
     protected HConnection connection;
 
     /**
@@ -852,12 +886,14 @@ public class PerformanceEvaluation exten
       this.startRow = options.getStartRow();
       this.perClientRunRows = options.getPerClientRunRows();
       this.totalRows = options.getTotalRows();
+      this.sampleRate = options.getSampleRate();
       this.status = status;
       this.tableName = options.getTableName();
       this.table = null;
       this.conf = conf;
       this.flushCommits = options.isFlushCommits();
       this.writeToWAL = options.isWriteToWAL();
+      this.reportLatency = options.isReportLatency();
       this.connection = options.getConnection();
     }
 
@@ -867,7 +903,7 @@ public class PerformanceEvaluation exten
 
     protected int getReportingPeriod() {
       int period = this.perClientRunRows / 10;
-      return period == 0? this.perClientRunRows: period;
+      return period == 0 ? this.perClientRunRows : period;
     }
 
     void testSetup() throws IOException {
@@ -933,17 +969,14 @@ public class PerformanceEvaluation exten
       scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
       scan.setFilter(new WhileMatchFilter(new PageFilter(120)));
       ResultScanner s = this.table.getScanner(scan);
-      //int count = 0;
-      for (Result rr = null; (rr = s.next()) != null;) {
-        // LOG.info("" + count++ + " " + rr.toString());
-      }
+      for (Result rr; (rr = s.next()) != null;) ;
       s.close();
     }
 
     @Override
     protected int getReportingPeriod() {
       int period = this.perClientRunRows / 100;
-      return period == 0? this.perClientRunRows: period;
+      return period == 0 ? this.perClientRunRows : period;
     }
 
   }
@@ -961,7 +994,7 @@ public class PerformanceEvaluation exten
       scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
       ResultScanner s = this.table.getScanner(scan);
       int count = 0;
-      for (Result rr = null; (rr = s.next()) != null;) {
+      for (Result rr; (rr = s.next()) != null;) {
         count++;
       }
 
@@ -1034,23 +1067,49 @@ public class PerformanceEvaluation exten
   }
 
   static class RandomReadTest extends Test {
+    private final int everyN;
+    private final boolean reportLatency;
+    private final float[] times;
+    int idx = 0;
+
     RandomReadTest(Configuration conf, TestOptions options, Status status) {
       super(conf, options, status);
+      everyN = (int) (this.totalRows / (this.totalRows * this.sampleRate));
+      LOG.info("Sampling 1 every " + everyN + " out of " + perClientRunRows + " total rows.");
+      this.reportLatency = options.isReportLatency();
+      if (this.reportLatency) {
+        times = new float[(int) Math.ceil(this.perClientRunRows * this.sampleRate)];
+      } else {
+        times = null;
+      }
     }
 
     @Override
     void testRow(final int i) throws IOException {
-      Get get = new Get(getRandomRow(this.rand, this.totalRows));
-      get.addColumn(FAMILY_NAME, QUALIFIER_NAME);
-      this.table.get(get);
+      if (i % everyN == 0) {
+        Get get = new Get(getRandomRow(this.rand, this.totalRows));
+        get.addColumn(FAMILY_NAME, QUALIFIER_NAME);
+        long start = System.nanoTime();
+        this.table.get(get);
+        if (this.reportLatency) {
+          times[idx++] = (float) ((System.nanoTime() - start) / 1000000.0);
+        }
+      }
     }
 
     @Override
     protected int getReportingPeriod() {
       int period = this.perClientRunRows / 100;
-      return period == 0? this.perClientRunRows: period;
+      return period == 0 ? this.perClientRunRows : period;
     }
 
+    @Override
+    protected void testTakedown() throws IOException {
+      super.testTakedown();
+      if (this.reportLatency) {
+        LOG.info("randomRead latency log (ms): " + Arrays.toString(times));
+      }
+    }
   }
 
   static class RandomWriteTest extends Test {
@@ -1110,7 +1169,6 @@ public class PerformanceEvaluation exten
       get.addColumn(FAMILY_NAME, QUALIFIER_NAME);
       table.get(get);
     }
-
   }
 
   static class SequentialWriteTest extends Test {
@@ -1126,7 +1184,6 @@ public class PerformanceEvaluation exten
       put.setDurability(writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
       table.put(put);
     }
-
   }
 
   static class FilteredScanTest extends Test {
@@ -1162,14 +1219,31 @@ public class PerformanceEvaluation exten
     }
   }
 
+  /**
+   * Compute a throughput rate in MB/s.
+   * @param rows Number of records consumed.
+   * @param timeMs Time taken in milliseconds.
+   * @return String value with label, ie '123.76 MB/s'
+   */
+  private static String calculateMbps(int rows, long timeMs) {
+    // MB/s = ((totalRows * ROW_SIZE_BYTES) / totalTimeMS)
+    //        * 1000 MS_PER_SEC / (1024 * 1024) BYTES_PER_MB
+    BigDecimal rowSize =
+      BigDecimal.valueOf(ROW_LENGTH + VALUE_LENGTH + FAMILY_NAME.length + QUALIFIER_NAME.length);
+    BigDecimal mbps = BigDecimal.valueOf(rows).multiply(rowSize, CXT)
+      .divide(BigDecimal.valueOf(timeMs), CXT).multiply(MS_PER_SEC, CXT)
+      .divide(BYTES_PER_MB, CXT);
+    return FMT.format(mbps) + " MB/s";
+  }
+
   /*
    * Format passed integer.
    * @param number
-   * @return Returns zero-prefixed 10-byte wide decimal version of passed
+   * @return Returns zero-prefixed ROW_LENGTH-byte wide decimal version of passed
    * number (Does absolute in case number is negative).
    */
   public static byte [] format(final int number) {
-    byte [] b = new byte[DEFAULT_ROW_PREFIX_LENGTH + 10];
+    byte [] b = new byte[ROW_LENGTH];
     int d = Math.abs(number);
     for (int i = b.length - 1; i >= 0; i--) {
       b[i] = (byte)((d % 10) + '0');
@@ -1211,16 +1285,17 @@ public class PerformanceEvaluation exten
   }
 
   long runOneClient(final Class<? extends Test> cmd, final int startRow,
-                    final int perClientRunRows, final int totalRows,
-                    boolean flushCommits, boolean writeToWAL, HConnection connection,
-                    final Status status)
+      final int perClientRunRows, final int totalRows, final float sampleRate,
+      boolean flushCommits, boolean writeToWAL, boolean reportLatency,
+      HConnection connection, final Status status)
   throws IOException {
     status.setStatus("Start " + cmd + " at offset " + startRow + " for " +
       perClientRunRows + " rows");
     long totalElapsedTime = 0;
 
     TestOptions options = new TestOptions(startRow, perClientRunRows,
-        totalRows, N, tableName, flushCommits, writeToWAL, connection);
+      totalRows, sampleRate, N, tableName, flushCommits, writeToWAL,
+      reportLatency, connection);
     final Test t;
     try {
       Constructor<? extends Test> constructor = cmd.getDeclaredConstructor(
@@ -1237,11 +1312,12 @@ public class PerformanceEvaluation exten
     totalElapsedTime = t.test();
 
     status.setStatus("Finished " + cmd + " in " + totalElapsedTime +
-      "ms at offset " + startRow + " for " + perClientRunRows + " rows");
+      "ms at offset " + startRow + " for " + perClientRunRows + " rows" +
+      " (" + calculateMbps((int)(perClientRunRows * sampleRate), totalElapsedTime) + ")");
     return totalElapsedTime;
   }
 
-  private void runNIsOne(final Class<? extends Test> cmd) {
+  private void runNIsOne(final Class<? extends Test> cmd) throws IOException {
     Status status = new Status() {
       public void setStatus(String msg) throws IOException {
         LOG.info(msg);
@@ -1252,10 +1328,12 @@ public class PerformanceEvaluation exten
     try {
       admin = new HBaseAdmin(getConf());
       checkTable(admin);
-      runOneClient(cmd, 0, this.R, this.R, this.flushCommits, this.writeToWAL, this.connection,
-        status);
+      runOneClient(cmd, 0, this.R, this.R, this.sampleRate, this.flushCommits,
+        this.writeToWAL, this.reportLatency, this.connection, status);
     } catch (Exception e) {
       LOG.error("Failed", e);
+    } finally {
+      if (admin != null) admin.close();
     }
   }
 
@@ -1281,20 +1359,27 @@ public class PerformanceEvaluation exten
     }
     System.err.println("Usage: java " + this.getClass().getName() + " \\");
     System.err.println("  [--nomapred] [--rows=ROWS] [--table=NAME] \\");
-    System.err.println("  [--compress=TYPE] [--blockEncoding=TYPE] [-D<property=value>]* <command> <nclients>");
+    System.err.println("  [--compress=TYPE] [--blockEncoding=TYPE] " +
+      "[-D<property=value>]* <command> <nclients>");
     System.err.println();
     System.err.println("Options:");
     System.err.println(" nomapred        Run multiple clients using threads " +
       "(rather than use mapreduce)");
     System.err.println(" rows            Rows each client runs. Default: One million");
+    System.err.println(" sampleRate      Execute test on a sample of total " +
+      "rows. Only supported by randomRead. Default: 1.0");
     System.err.println(" table           Alternate table name. Default: 'TestTable'");
     System.err.println(" compress        Compression type to use (GZ, LZO, ...). Default: 'NONE'");
-    System.err.println(" flushCommits    Used to determine if the test should flush the table.  Default: false");
+    System.err.println(" flushCommits    Used to determine if the test should flush the table. " +
+      "Default: false");
     System.err.println(" writeToWAL      Set writeToWAL on puts. Default: True");
-    System.err.println(" presplit        Create presplit table. Recommended for accurate perf analysis (see guide).  Default: disabled");
-    System.err
-        .println(" inmemory        Tries to keep the HFiles of the CF inmemory as far as possible.  Not " +
-        		"guaranteed that reads are always served from inmemory.  Default: false");
+    System.err.println(" presplit        Create presplit table. Recommended for accurate perf " +
+      "analysis (see guide).  Default: disabled");
+    System.err.println(" inmemory        Tries to keep the HFiles of the CF " +
+      "inmemory as far as possible. Not guaranteed that reads are always served " +
+      "from memory.  Default: false");
+    System.err.println(" latency         Set to report operation latencies. " +
+      "Currently only supported by randomRead test. Default: False");
     System.err.println();
     System.err.println(" Note: -D properties will be applied to the conf used. ");
     System.err.println("  For example: ");
@@ -1364,6 +1449,12 @@ public class PerformanceEvaluation exten
           continue;
         }
 
+        final String sampleRate = "--sampleRate=";
+        if (cmd.startsWith(sampleRate)) {
+          this.sampleRate = Float.parseFloat(cmd.substring(sampleRate.length()));
+          continue;
+        }
+
         final String table = "--table=";
         if (cmd.startsWith(table)) {
           this.tableName = TableName.valueOf(cmd.substring(table.length()));
@@ -1406,6 +1497,12 @@ public class PerformanceEvaluation exten
           continue;
         }
 
+        final String latency = "--latency";
+        if (cmd.startsWith(latency)) {
+          this.reportLatency = true;
+          continue;
+        }
+
         this.connection = HConnectionManager.createConnection(getConf());
         
         Class<? extends Test> cmdClass = determineCommandClass(cmd);
@@ -1431,9 +1528,6 @@ public class PerformanceEvaluation exten
     return descriptor != null ? descriptor.getCmdClass() : null;
   }
 
-  /**
-   * @param args
-   */
   public static void main(final String[] args) throws Exception {
     int res = ToolRunner.run(new PerformanceEvaluation(HBaseConfiguration.create()), args);
     System.exit(res);