1   /**
2    * Copyright 2007 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase;
21  
22  import java.io.DataInput;
23  import java.io.DataOutput;
24  import java.io.File;
25  import java.io.IOException;
26  import java.io.PrintStream;
27  import java.math.BigDecimal;
28  import java.math.MathContext;
29  import java.text.DecimalFormat;
30  import java.text.SimpleDateFormat;
31  import java.util.ArrayList;
32  import java.util.Date;
33  import java.util.List;
34  import java.util.Map;
35  import java.util.Random;
36  import java.util.TreeMap;
37  import java.util.Arrays;
38  import java.util.regex.Matcher;
39  import java.util.regex.Pattern;
40  import java.lang.reflect.Constructor;
41  
42  import org.apache.commons.logging.Log;
43  import org.apache.commons.logging.LogFactory;
44  import org.apache.hadoop.conf.Configuration;
45  import org.apache.hadoop.fs.FSDataInputStream;
46  import org.apache.hadoop.fs.FileStatus;
47  import org.apache.hadoop.fs.FileSystem;
48  import org.apache.hadoop.fs.Path;
49  import org.apache.hadoop.hbase.client.Get;
50  import org.apache.hadoop.hbase.client.HBaseAdmin;
51  import org.apache.hadoop.hbase.client.HTable;
52  import org.apache.hadoop.hbase.client.Put;
53  import org.apache.hadoop.hbase.client.Result;
54  import org.apache.hadoop.hbase.client.ResultScanner;
55  import org.apache.hadoop.hbase.client.Scan;
56  import org.apache.hadoop.hbase.filter.PageFilter;
57  import org.apache.hadoop.hbase.filter.WhileMatchFilter;
58  import org.apache.hadoop.hbase.filter.Filter;
59  import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
60  import org.apache.hadoop.hbase.filter.CompareFilter;
61  import org.apache.hadoop.hbase.filter.BinaryComparator;
62  import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
63  import org.apache.hadoop.hbase.util.Bytes;
64  import org.apache.hadoop.hbase.util.FSUtils;
65  import org.apache.hadoop.hbase.util.Hash;
66  import org.apache.hadoop.hbase.util.MurmurHash;
67  import org.apache.hadoop.hbase.util.Pair;
68  import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
69  import org.apache.hadoop.hdfs.MiniDFSCluster;
70  import org.apache.hadoop.io.LongWritable;
71  import org.apache.hadoop.io.NullWritable;
72  import org.apache.hadoop.io.Text;
73  import org.apache.hadoop.io.Writable;
74  import org.apache.hadoop.mapreduce.InputSplit;
75  import org.apache.hadoop.mapreduce.Job;
76  import org.apache.hadoop.mapreduce.JobContext;
77  import org.apache.hadoop.mapreduce.Mapper;
78  import org.apache.hadoop.mapreduce.RecordReader;
79  import org.apache.hadoop.mapreduce.TaskAttemptContext;
80  import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
81  import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
82  import org.apache.hadoop.mapreduce.lib.reduce.LongSumReducer;
83  import org.apache.hadoop.util.LineReader;
84  
85  /**
86   * Script used evaluating HBase performance and scalability.  Runs a HBase
87   * client that steps through one of a set of hardcoded tests or 'experiments'
88   * (e.g. a random reads test, a random writes test, etc.). Pass on the
89   * command-line which test to run and how many clients are participating in
90   * this experiment. Run <code>java PerformanceEvaluation --help</code> to
91   * obtain usage.
92   *
93   * <p>This class sets up and runs the evaluation programs described in
94   * Section 7, <i>Performance Evaluation</i>, of the <a
95   * href="http://labs.google.com/papers/bigtable.html">Bigtable</a>
96   * paper, pages 8-10.
97   *
98   * <p>If number of clients > 1, we start up a MapReduce job. Each map task
99   * runs an individual client. Each client does about 1GB of data.
100  */
101 public class PerformanceEvaluation {
102   protected static final Log LOG = LogFactory.getLog(PerformanceEvaluation.class.getName());
103 
104   public static final byte[] TABLE_NAME = Bytes.toBytes("TestTable");
105   public static final byte[] FAMILY_NAME = Bytes.toBytes("info");
106   public static final byte[] QUALIFIER_NAME = Bytes.toBytes("data");
107   public static final int VALUE_LENGTH = 1000;
108   public static final int ROW_LENGTH = 26;
109 
110   private static final int ONE_GB = 1024 * 1024 * 1000;
111   private static final int ROWS_PER_GB = ONE_GB / VALUE_LENGTH;
112   private static final DecimalFormat FMT = new DecimalFormat("0.##");
113   private static final MathContext CXT = MathContext.DECIMAL64;
114   private static final BigDecimal MS_PER_SEC = BigDecimal.valueOf(1000);
115   private static final BigDecimal BYTES_PER_MB = BigDecimal.valueOf(1024 * 1024);
116 
117   protected static HTableDescriptor TABLE_DESCRIPTOR;
118   protected Map<String, CmdDescriptor> commands = new TreeMap<String, CmdDescriptor>();
119 
120   volatile Configuration conf;
121   private boolean miniCluster = false;
122   private boolean nomapred = false;
123   private int N = 1;
124   private int R = ROWS_PER_GB;
125   private float sampleRate = 1.0f;
126   private boolean flushCommits = true;
127   private boolean reportLatency = false;
128   private boolean writeToWAL = true;
129   private int presplitRegions = 0;
130 
131   private static final Path PERF_EVAL_DIR = new Path("performance_evaluation");
132 
133   /** Regex to parse lines in input file passed to mapreduce task. */
134   public static final Pattern LINE_PATTERN =
135     Pattern.compile("startRow=(\\d+),\\s+" +
136         "perClientRunRows=(\\d+),\\s+" +
137         "totalRows=(\\d+),\\s+" +
138         "sampleRate=([-+]?[0-9]*\\.?[0-9]+),\\s+" +
139         "clients=(\\d+),\\s+" +
140         "flushCommits=(\\w+),\\s+" +
141         "writeToWAL=(\\w+),\\s+" +
142         "reportLatency=(\\w+)");
143 
144   /**
145    * Enum for map metrics.  Keep it out here rather than inside in the Map
146    * inner-class so we can find associated properties.
147    */
148   protected static enum Counter {
149     /** elapsed time */
150     ELAPSED_TIME,
151     /** number of rows */
152     ROWS
153   }
154 
155   /**
156    * Constructor
157    * @param c Configuration object
158    */
159   public PerformanceEvaluation(final Configuration c) {
160     this.conf = c;
161 
162     addCommandDescriptor(RandomReadTest.class, "randomRead",
163         "Run random read test");
164     addCommandDescriptor(RandomSeekScanTest.class, "randomSeekScan",
165         "Run random seek and scan 100 test");
166     addCommandDescriptor(RandomScanWithRange10Test.class, "scanRange10",
167         "Run random seek scan with both start and stop row (max 10 rows)");
168     addCommandDescriptor(RandomScanWithRange100Test.class, "scanRange100",
169         "Run random seek scan with both start and stop row (max 100 rows)");
170     addCommandDescriptor(RandomScanWithRange1000Test.class, "scanRange1000",
171         "Run random seek scan with both start and stop row (max 1000 rows)");
172     addCommandDescriptor(RandomScanWithRange10000Test.class, "scanRange10000",
173         "Run random seek scan with both start and stop row (max 10000 rows)");
174     addCommandDescriptor(RandomWriteTest.class, "randomWrite",
175         "Run random write test");
176     addCommandDescriptor(SequentialReadTest.class, "sequentialRead",
177         "Run sequential read test");
178     addCommandDescriptor(SequentialWriteTest.class, "sequentialWrite",
179         "Run sequential write test");
180     addCommandDescriptor(ScanTest.class, "scan",
181         "Run scan test (read every row)");
182     addCommandDescriptor(FilteredScanTest.class, "filterScan",
183         "Run scan test using a filter to find a specific row based on it's value (make sure to use --rows=20)");
184   }
185 
186   protected void addCommandDescriptor(Class<? extends Test> cmdClass,
187       String name, String description) {
188     CmdDescriptor cmdDescriptor =
189       new CmdDescriptor(cmdClass, name, description);
190     commands.put(name, cmdDescriptor);
191   }
192 
193   /**
194    * Implementations can have their status set.
195    */
196   static interface Status {
197     /**
198      * Sets status
199      * @param msg status message
200      * @throws IOException
201      */
202     void setStatus(final String msg) throws IOException;
203   }
204 
205   /**
206    *  This class works as the InputSplit of Performance Evaluation
207    *  MapReduce InputFormat, and the Record Value of RecordReader.
208    *  Each map task will only read one record from a PeInputSplit,
209    *  the record value is the PeInputSplit itself.
210    */
211   public static class PeInputSplit extends InputSplit implements Writable {
212     private int startRow = 0;
213     private int rows = 0;
214     private int totalRows = 0;
215     private float sampleRate = 1.0f;
216     private int clients = 0;
217     private boolean flushCommits = false;
218     private boolean writeToWAL = true;
219     private boolean reportLatency = false;
220 
221     public PeInputSplit() {}
222 
223     public PeInputSplit(int startRow, int rows, int totalRows, float sampleRate,
224         int clients, boolean flushCommits, boolean writeToWAL, boolean reportLatency) {
225       this.startRow = startRow;
226       this.rows = rows;
227       this.totalRows = totalRows;
228       this.sampleRate = sampleRate;
229       this.clients = clients;
230       this.flushCommits = flushCommits;
231       this.writeToWAL = writeToWAL;
232       this.reportLatency = reportLatency;
233     }
234 
235     @Override
236     public void readFields(DataInput in) throws IOException {
237       this.startRow = in.readInt();
238       this.rows = in.readInt();
239       this.totalRows = in.readInt();
240       this.sampleRate = in.readFloat();
241       this.clients = in.readInt();
242       this.flushCommits = in.readBoolean();
243       this.writeToWAL = in.readBoolean();
244       this.reportLatency = in.readBoolean();
245     }
246 
247     @Override
248     public void write(DataOutput out) throws IOException {
249       out.writeInt(startRow);
250       out.writeInt(rows);
251       out.writeInt(totalRows);
252       out.writeFloat(sampleRate);
253       out.writeInt(clients);
254       out.writeBoolean(flushCommits);
255       out.writeBoolean(writeToWAL);
256       out.writeBoolean(reportLatency);
257     }
258 
259     @Override
260     public long getLength() throws IOException, InterruptedException {
261       return 0;
262     }
263 
264     @Override
265     public String[] getLocations() throws IOException, InterruptedException {
266       return new String[0];
267     }
268 
269     public int getStartRow() {
270       return startRow;
271     }
272 
273     public int getRows() {
274       return rows;
275     }
276 
277     public int getTotalRows() {
278       return totalRows;
279     }
280 
281     public float getSampleRate() {
282       return sampleRate;
283     }
284 
285     public int getClients() {
286       return clients;
287     }
288 
289     public boolean isFlushCommits() {
290       return flushCommits;
291     }
292 
293     public boolean isWriteToWAL() {
294       return writeToWAL;
295     }
296 
297     public boolean isReportLatency() {
298       return reportLatency;
299     }
300   }
301 
302   /**
303    *  InputFormat of Performance Evaluation MapReduce job.
304    *  It extends from FileInputFormat, want to use it's methods such as setInputPaths().
305    */
306   public static class PeInputFormat extends FileInputFormat<NullWritable, PeInputSplit> {
307 
308     @Override
309     public List<InputSplit> getSplits(JobContext job) throws IOException {
310       // generate splits
311       List<InputSplit> splitList = new ArrayList<InputSplit>();
312 
313       for (FileStatus file: listStatus(job)) {
314         Path path = file.getPath();
315         FileSystem fs = path.getFileSystem(job.getConfiguration());
316         FSDataInputStream fileIn = fs.open(path);
317         LineReader in = new LineReader(fileIn, job.getConfiguration());
318         int lineLen = 0;
319         while(true) {
320           Text lineText = new Text();
321           lineLen = in.readLine(lineText);
322           if(lineLen <= 0) {
323           break;
324           }
325           Matcher m = LINE_PATTERN.matcher(lineText.toString());
326           if((m != null) && m.matches()) {
327             int startRow = Integer.parseInt(m.group(1));
328             int rows = Integer.parseInt(m.group(2));
329             int totalRows = Integer.parseInt(m.group(3));
330             float sampleRate = Float.parseFloat(m.group(4));
331             int clients = Integer.parseInt(m.group(5));
332             boolean flushCommits = Boolean.parseBoolean(m.group(6));
333             boolean writeToWAL = Boolean.parseBoolean(m.group(7));
334             boolean reportLatency = Boolean.parseBoolean(m.group(8));
335 
336             LOG.debug("split["+ splitList.size() + "] " +
337                      " startRow=" + startRow +
338                      " rows=" + rows +
339                      " totalRows=" + totalRows +
340                      " sampleRate=" + sampleRate +
341                      " clients=" + clients +
342                      " flushCommits=" + flushCommits +
343                      " writeToWAL=" + writeToWAL +
344                      " reportLatency=" + reportLatency);
345 
346             PeInputSplit newSplit =
347               new PeInputSplit(startRow, rows, totalRows, sampleRate, clients,
348                 flushCommits, writeToWAL, reportLatency);
349             splitList.add(newSplit);
350           }
351         }
352         in.close();
353       }
354 
355       LOG.info("Total # of splits: " + splitList.size());
356       return splitList;
357     }
358 
359     @Override
360     public RecordReader<NullWritable, PeInputSplit> createRecordReader(InputSplit split,
361                             TaskAttemptContext context) {
362       return new PeRecordReader();
363     }
364 
365     public static class PeRecordReader extends RecordReader<NullWritable, PeInputSplit> {
366       private boolean readOver = false;
367       private PeInputSplit split = null;
368       private NullWritable key = null;
369       private PeInputSplit value = null;
370 
371       @Override
372       public void initialize(InputSplit split, TaskAttemptContext context)
373                   throws IOException, InterruptedException {
374         this.readOver = false;
375         this.split = (PeInputSplit)split;
376       }
377 
378       @Override
379       public boolean nextKeyValue() throws IOException, InterruptedException {
380         if(readOver) {
381           return false;
382         }
383 
384         key = NullWritable.get();
385         value = split;
386 
387         readOver = true;
388         return true;
389       }
390 
391       @Override
392       public NullWritable getCurrentKey() throws IOException, InterruptedException {
393         return key;
394       }
395 
396       @Override
397       public PeInputSplit getCurrentValue() throws IOException, InterruptedException {
398         return value;
399       }
400 
401       @Override
402       public float getProgress() throws IOException, InterruptedException {
403         if(readOver) {
404           return 1.0f;
405         } else {
406           return 0.0f;
407         }
408       }
409 
410       @Override
411       public void close() throws IOException {
412         // do nothing
413       }
414     }
415   }
416 
417   /**
418    * MapReduce job that runs a performance evaluation client in each map task.
419    */
420   public static class EvaluationMapTask
421       extends Mapper<NullWritable, PeInputSplit, LongWritable, LongWritable> {
422 
423     /** configuration parameter name that contains the command */
424     public final static String CMD_KEY = "EvaluationMapTask.command";
425     /** configuration parameter name that contains the PE impl */
426     public static final String PE_KEY = "EvaluationMapTask.performanceEvalImpl";
427 
428     private Class<? extends Test> cmd;
429     private PerformanceEvaluation pe;
430 
431     @Override
432     protected void setup(Context context) throws IOException, InterruptedException {
433       this.cmd = forName(context.getConfiguration().get(CMD_KEY), Test.class);
434 
435       // this is required so that extensions of PE are instantiated within the
436       // map reduce task...
437       Class<? extends PerformanceEvaluation> peClass =
438           forName(context.getConfiguration().get(PE_KEY), PerformanceEvaluation.class);
439       try {
440         this.pe = peClass.getConstructor(Configuration.class)
441             .newInstance(context.getConfiguration());
442       } catch (Exception e) {
443         throw new IllegalStateException("Could not instantiate PE instance", e);
444       }
445     }
446 
447     private <Type> Class<? extends Type> forName(String className, Class<Type> type) {
448       Class<? extends Type> clazz = null;
449       try {
450         clazz = Class.forName(className).asSubclass(type);
451       } catch (ClassNotFoundException e) {
452         throw new IllegalStateException("Could not find class for name: " + className, e);
453       }
454       return clazz;
455     }
456 
457     protected void map(NullWritable key, PeInputSplit value, final Context context)
458            throws IOException, InterruptedException {
459 
460       Status status = new Status() {
461         public void setStatus(String msg) {
462            context.setStatus(msg);
463         }
464       };
465 
466       // Evaluation task
467       long elapsedTime = this.pe.runOneClient(this.cmd, value.getStartRow(),
468         value.getRows(), value.getTotalRows(), value.getSampleRate(),
469         value.isFlushCommits(), value.isWriteToWAL(), value.isReportLatency(),
470         status);
471       // Collect how much time the thing took. Report as map output and
472       // to the ELAPSED_TIME counter.
473       context.getCounter(Counter.ELAPSED_TIME).increment(elapsedTime);
474       context.getCounter(Counter.ROWS).increment(value.rows);
475       context.write(new LongWritable(value.startRow), new LongWritable(elapsedTime));
476       context.progress();
477     }
478   }
479 
480   /*
481    * If table does not already exist, create.
482    * @param c Client to use checking.
483    * @return True if we created the table.
484    * @throws IOException
485    */
486   private boolean checkTable(HBaseAdmin admin) throws IOException {
487     HTableDescriptor tableDescriptor = getTableDescriptor();
488     if (this.presplitRegions > 0) {
489       // presplit requested
490       if (admin.tableExists(tableDescriptor.getName())) {
491         admin.disableTable(tableDescriptor.getName());
492         admin.deleteTable(tableDescriptor.getName());
493       }
494 
495       byte[][] splits = getSplits();
496       for (int i=0; i < splits.length; i++) {
497         LOG.debug(" split " + i + ": " + Bytes.toStringBinary(splits[i]));
498       }
499       admin.createTable(tableDescriptor, splits);
500       LOG.info ("Table created with " + this.presplitRegions + " splits");
501     }
502     else {
503       boolean tableExists = admin.tableExists(tableDescriptor.getName());
504       if (!tableExists) {
505         admin.createTable(tableDescriptor);
506         LOG.info("Table " + tableDescriptor + " created");
507       }
508     }
509     return admin.tableExists(tableDescriptor.getName());
510   }
511 
512   protected HTableDescriptor getTableDescriptor() {
513     if (TABLE_DESCRIPTOR == null) {
514       TABLE_DESCRIPTOR = new HTableDescriptor(TABLE_NAME);
515       HColumnDescriptor family = new HColumnDescriptor(FAMILY_NAME);
516       TABLE_DESCRIPTOR.addFamily(family);
517     }
518     return TABLE_DESCRIPTOR;
519   }
520 
521   /**
522    * generates splits based on total number of rows and specified split regions
523    *
524    * @return splits : array of byte []
525    */
526   protected  byte[][] getSplits() {
527     if (this.presplitRegions == 0)
528       return new byte [0][];
529 
530     byte[][] splits = new byte[this.presplitRegions][];
531     int jump = this.R  / this.presplitRegions;
532     for (int i=0; i <this.presplitRegions; i++) {
533       int rowkey = jump * i;
534       splits[i] = format(rowkey);
535     }
536     return splits;
537   }
538 
539   /*
540    * We're to run multiple clients concurrently.  Setup a mapreduce job.  Run
541    * one map per client.  Then run a single reduce to sum the elapsed times.
542    * @param cmd Command to run.
543    * @throws IOException
544    */
545   private void runNIsMoreThanOne(final Class<? extends Test> cmd)
546   throws IOException, InterruptedException, ClassNotFoundException {
547     checkTable(new HBaseAdmin(conf));
548     if (this.nomapred) {
549       doMultipleClients(cmd);
550     } else {
551       doMapReduce(cmd);
552     }
553   }
554 
555   /*
556    * Run all clients in this vm each to its own thread.
557    * @param cmd Command to run.
558    * @throws IOException
559    */
560   private void doMultipleClients(final Class<? extends Test> cmd) throws IOException {
561     final List<Thread> threads = new ArrayList<Thread>(this.N);
562     final int perClientRows = R/N;
563     for (int i = 0; i < this.N; i++) {
564       Thread t = new Thread (Integer.toString(i)) {
565         @Override
566         public void run() {
567           super.run();
568           PerformanceEvaluation pe = new PerformanceEvaluation(conf);
569           int index = Integer.parseInt(getName());
570           try {
571             long elapsedTime = pe.runOneClient(cmd, index * perClientRows,
572                perClientRows, R, sampleRate, flushCommits, writeToWAL,
573                reportLatency, new Status() {
574                   public void setStatus(final String msg) throws IOException {
575                     LOG.info("client-" + getName() + " " + msg);
576                   }
577                 });
578             LOG.info("Finished " + getName() + " in " + elapsedTime +
579               "ms writing " + perClientRows + " rows");
580           } catch (IOException e) {
581             throw new RuntimeException(e);
582           }
583         }
584       };
585       threads.add(t);
586     }
587     for (Thread t: threads) {
588       t.start();
589     }
590     for (Thread t: threads) {
591       while(t.isAlive()) {
592         try {
593           t.join();
594         } catch (InterruptedException e) {
595           LOG.debug("Interrupted, continuing" + e.toString());
596         }
597       }
598     }
599   }
600 
601   /*
602    * Run a mapreduce job.  Run as many maps as asked-for clients.
603    * Before we start up the job, write out an input file with instruction
604    * per client regards which row they are to start on.
605    * @param cmd Command to run.
606    * @throws IOException
607    */
608   private void doMapReduce(final Class<? extends Test> cmd) throws IOException,
609         InterruptedException, ClassNotFoundException {
610     Path inputDir = writeInputFile(this.conf);
611     this.conf.set(EvaluationMapTask.CMD_KEY, cmd.getName());
612     this.conf.set(EvaluationMapTask.PE_KEY, getClass().getName());
613     Job job = new Job(this.conf);
614     job.setJarByClass(PerformanceEvaluation.class);
615     job.setJobName("HBase Performance Evaluation");
616 
617     job.setInputFormatClass(PeInputFormat.class);
618     PeInputFormat.setInputPaths(job, inputDir);
619 
620     job.setOutputKeyClass(LongWritable.class);
621     job.setOutputValueClass(LongWritable.class);
622 
623     job.setMapperClass(EvaluationMapTask.class);
624     job.setReducerClass(LongSumReducer.class);
625 
626     job.setNumReduceTasks(1);
627 
628     job.setOutputFormatClass(TextOutputFormat.class);
629     TextOutputFormat.setOutputPath(job, new Path(inputDir,"outputs"));
630 
631     TableMapReduceUtil.addDependencyJars(job);
632     // Add a Class from the hbase.jar so it gets registered too.
633     TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
634       org.apache.hadoop.hbase.util.Bytes.class);
635 
636     TableMapReduceUtil.initCredentials(job);
637 
638     job.waitForCompletion(true);
639   }
640 
641   /*
642    * Write input file of offsets-per-client for the mapreduce job.
643    * @param c Configuration
644    * @return Directory that contains file written.
645    * @throws IOException
646    */
647   private Path writeInputFile(final Configuration c) throws IOException {
648     FileSystem fs = FileSystem.get(c);
649     if (!fs.exists(PERF_EVAL_DIR)) {
650       fs.mkdirs(PERF_EVAL_DIR);
651     }
652     SimpleDateFormat formatter = new SimpleDateFormat("yyyyMMddHHmmss");
653     Path subdir = new Path(PERF_EVAL_DIR, formatter.format(new Date()));
654     fs.mkdirs(subdir);
655     Path inputFile = new Path(subdir, "input.txt");
656     PrintStream out = new PrintStream(fs.create(inputFile));
657     // Make input random.
658     Map<Integer, String> m = new TreeMap<Integer, String>();
659     Hash h = MurmurHash.getInstance();
660     int perClientRows = (this.R / this.N);
661     try {
662       for (int i = 0; i < 10; i++) {
663         for (int j = 0; j < N; j++) {
664           String s = "startRow=" + ((j * perClientRows) + (i * (perClientRows/10))) +
665           ", perClientRunRows=" + (perClientRows / 10) +
666           ", totalRows=" + this.R +
667           ", sampleRate=" + this.sampleRate +
668           ", clients=" + this.N +
669           ", flushCommits=" + this.flushCommits +
670           ", writeToWAL=" + this.writeToWAL +
671           ", reportLatency=" + this.reportLatency;
672           int hash = h.hash(Bytes.toBytes(s));
673           m.put(hash, s);
674         }
675       }
676       for (Map.Entry<Integer, String> e: m.entrySet()) {
677         out.println(e.getValue());
678       }
679     } finally {
680       out.close();
681     }
682     return subdir;
683   }
684 
685   /**
686    * Describes a command.
687    */
688   static class CmdDescriptor {
689     private Class<? extends Test> cmdClass;
690     private String name;
691     private String description;
692 
693     CmdDescriptor(Class<? extends Test> cmdClass, String name, String description) {
694       this.cmdClass = cmdClass;
695       this.name = name;
696       this.description = description;
697     }
698 
699     public Class<? extends Test> getCmdClass() {
700       return cmdClass;
701     }
702 
703     public String getName() {
704       return name;
705     }
706 
707     public String getDescription() {
708       return description;
709     }
710   }
711 
712   /**
713    * Wraps up options passed to {@link org.apache.hadoop.hbase.PerformanceEvaluation.Test
714    * tests}.  This makes the reflection logic a little easier to understand...
715    */
716   static class TestOptions {
717     private int startRow;
718     private int perClientRunRows;
719     private int totalRows;
720     private float sampleRate;
721     private byte[] tableName;
722     private boolean flushCommits;
723     private boolean writeToWAL = true;
724     private boolean reportLatency;
725 
726     TestOptions() {}
727 
728     TestOptions(int startRow, int perClientRunRows, int totalRows, float sampleRate,
729         byte[] tableName, boolean flushCommits, boolean writeToWAL, boolean reportLatency) {
730       this.startRow = startRow;
731       this.perClientRunRows = perClientRunRows;
732       this.totalRows = totalRows;
733       this.sampleRate = sampleRate;
734       this.tableName = tableName;
735       this.flushCommits = flushCommits;
736       this.writeToWAL = writeToWAL;
737       this.reportLatency = reportLatency;
738     }
739 
740     public int getStartRow() {
741       return startRow;
742     }
743 
744     public int getPerClientRunRows() {
745       return perClientRunRows;
746     }
747 
748     public int getTotalRows() {
749       return totalRows;
750     }
751 
752     public float getSampleRate() {
753       return sampleRate;
754     }
755 
756     public byte[] getTableName() {
757       return tableName;
758     }
759 
760     public boolean isFlushCommits() {
761       return flushCommits;
762     }
763 
764     public boolean isWriteToWAL() {
765       return writeToWAL;
766     }
767 
768     public boolean isReportLatency() {
769       return reportLatency;
770     }
771   }
772 
773   /*
774    * A test.
775    * Subclass to particularize what happens per row.
776    */
777   static abstract class Test {
778     // Below is make it so when Tests are all running in the one
779     // jvm, that they each have a differently seeded Random.
780     private static final Random randomSeed =
781       new Random(System.currentTimeMillis());
782     private static long nextRandomSeed() {
783       return randomSeed.nextLong();
784     }
785     protected final Random rand = new Random(nextRandomSeed());
786 
787     protected final int startRow;
788     protected final int perClientRunRows;
789     protected final int totalRows;
790     protected final float sampleRate;
791     private final Status status;
792     protected byte[] tableName;
793     protected HBaseAdmin admin;
794     protected HTable table;
795     protected volatile Configuration conf;
796     protected boolean flushCommits;
797     protected boolean writeToWAL;
798     protected boolean reportlatency;
799 
800     /**
801      * Note that all subclasses of this class must provide a public contructor
802      * that has the exact same list of arguments.
803      */
804     Test(final Configuration conf, final TestOptions options, final Status status) {
805       super();
806       this.startRow = options.getStartRow();
807       this.perClientRunRows = options.getPerClientRunRows();
808       this.totalRows = options.getTotalRows();
809       this.sampleRate = options.getSampleRate();
810       this.status = status;
811       this.tableName = options.getTableName();
812       this.table = null;
813       this.conf = conf;
814       this.flushCommits = options.isFlushCommits();
815       this.writeToWAL = options.isWriteToWAL();
816       this.reportlatency = options.isReportLatency();
817     }
818 
819     private String generateStatus(final int sr, final int i, final int lr) {
820       return sr + "/" + i + "/" + lr;
821     }
822 
823     protected int getReportingPeriod() {
824       int period = this.perClientRunRows / 10;
825       return period == 0 ? this.perClientRunRows : period;
826     }
827 
828     void testSetup() throws IOException {
829       this.admin = new HBaseAdmin(conf);
830       this.table = new HTable(conf, tableName);
831       this.table.setAutoFlush(false);
832       this.table.setScannerCaching(30);
833     }
834 
835     void testTakedown()  throws IOException {
836       if (flushCommits) {
837         this.table.flushCommits();
838       }
839       table.close();
840     }
841 
842     /*
843      * Run test
844      * @return Elapsed time.
845      * @throws IOException
846      */
847     long test() throws IOException {
848       long elapsedTime;
849       testSetup();
850       long startTime = System.currentTimeMillis();
851       try {
852         testTimed();
853         elapsedTime = System.currentTimeMillis() - startTime;
854       } finally {
855         testTakedown();
856       }
857       return elapsedTime;
858     }
859 
860     /**
861      * Provides an extension point for tests that don't want a per row invocation.
862      */
863     void testTimed() throws IOException {
864       int lastRow = this.startRow + this.perClientRunRows;
865       // Report on completion of 1/10th of total.
866       for (int i = this.startRow; i < lastRow; i++) {
867         testRow(i);
868         if (status != null && i > 0 && (i % getReportingPeriod()) == 0) {
869           status.setStatus(generateStatus(this.startRow, i, lastRow));
870         }
871       }
872     }
873 
874     /*
875     * Test for individual row.
876     * @param i Row index.
877     */
878     void testRow(final int i) throws IOException {
879     }
880   }
881 
882   @SuppressWarnings("unused")
883   static class RandomSeekScanTest extends Test {
884     RandomSeekScanTest(Configuration conf, TestOptions options, Status status) {
885       super(conf, options, status);
886     }
887 
888     @Override
889     void testRow(final int i) throws IOException {
890       Scan scan = new Scan(getRandomRow(this.rand, this.totalRows));
891       scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
892       scan.setFilter(new WhileMatchFilter(new PageFilter(120)));
893       ResultScanner s = this.table.getScanner(scan);
894       for (Result rr; (rr = s.next()) != null;) ;
895       s.close();
896     }
897 
898     @Override
899     protected int getReportingPeriod() {
900       int period = this.perClientRunRows / 100;
901       return period == 0 ? this.perClientRunRows : period;
902     }
903 
904   }
905 
906   @SuppressWarnings("unused")
907   static abstract class RandomScanWithRangeTest extends Test {
908     RandomScanWithRangeTest(Configuration conf, TestOptions options, Status status) {
909       super(conf, options, status);
910     }
911 
912     @Override
913     void testRow(final int i) throws IOException {
914       Pair<byte[], byte[]> startAndStopRow = getStartAndStopRow();
915       Scan scan = new Scan(startAndStopRow.getFirst(), startAndStopRow.getSecond());
916       scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
917       ResultScanner s = this.table.getScanner(scan);
918       int count = 0;
919       for (Result rr; (rr = s.next()) != null;) {
920         count++;
921       }
922 
923       if (i % 100 == 0) {
924         LOG.info(String.format("Scan for key range %s - %s returned %s rows",
925             Bytes.toString(startAndStopRow.getFirst()),
926             Bytes.toString(startAndStopRow.getSecond()), count));
927       }
928 
929       s.close();
930     }
931 
932     protected abstract Pair<byte[],byte[]> getStartAndStopRow();
933 
934     protected Pair<byte[], byte[]> generateStartAndStopRows(int maxRange) {
935       int start = this.rand.nextInt(Integer.MAX_VALUE) % totalRows;
936       int stop = start + maxRange;
937       return new Pair<byte[],byte[]>(format(start), format(stop));
938     }
939 
940     @Override
941     protected int getReportingPeriod() {
942       int period = this.perClientRunRows / 100;
943       return period == 0? this.perClientRunRows: period;
944     }
945   }
946 
947   static class RandomScanWithRange10Test extends RandomScanWithRangeTest {
948     RandomScanWithRange10Test(Configuration conf, TestOptions options, Status status) {
949       super(conf, options, status);
950     }
951 
952     @Override
953     protected Pair<byte[], byte[]> getStartAndStopRow() {
954       return generateStartAndStopRows(10);
955     }
956   }
957 
958   static class RandomScanWithRange100Test extends RandomScanWithRangeTest {
959     RandomScanWithRange100Test(Configuration conf, TestOptions options, Status status) {
960       super(conf, options, status);
961     }
962 
963     @Override
964     protected Pair<byte[], byte[]> getStartAndStopRow() {
965       return generateStartAndStopRows(100);
966     }
967   }
968 
969   static class RandomScanWithRange1000Test extends RandomScanWithRangeTest {
970     RandomScanWithRange1000Test(Configuration conf, TestOptions options, Status status) {
971       super(conf, options, status);
972     }
973 
974     @Override
975     protected Pair<byte[], byte[]> getStartAndStopRow() {
976       return generateStartAndStopRows(1000);
977     }
978   }
979 
980   static class RandomScanWithRange10000Test extends RandomScanWithRangeTest {
981     RandomScanWithRange10000Test(Configuration conf, TestOptions options, Status status) {
982       super(conf, options, status);
983     }
984 
985     @Override
986     protected Pair<byte[], byte[]> getStartAndStopRow() {
987       return generateStartAndStopRows(10000);
988     }
989   }
990 
991   static class RandomReadTest extends Test {
992     private final int everyN;
993     private final boolean reportLatency;
994     private final float[] times;
995     int idx = 0;
996 
997     RandomReadTest(Configuration conf, TestOptions options, Status status) {
998       super(conf, options, status);
999       everyN = (int) (this.totalRows / (this.totalRows * this.sampleRate));
1000       LOG.info("Sampling 1 every " + everyN + " out of " + perClientRunRows + " total rows.");
1001       this.reportLatency = options.isReportLatency();
1002       if (this.reportLatency) {
1003         times = new float[(int) Math.ceil(this.perClientRunRows * this.sampleRate)];
1004       } else {
1005         times = null;
1006       }
1007     }
1008 
1009     @Override
1010     void testRow(final int i) throws IOException {
1011       if (i % everyN == 0) {
1012         Get get = new Get(getRandomRow(this.rand, this.totalRows));
1013         get.addColumn(FAMILY_NAME, QUALIFIER_NAME);
1014         long start = System.nanoTime();
1015         this.table.get(get);
1016         if (this.reportLatency) {
1017           times[idx++] = (float) ((System.nanoTime() - start) / 1000000.0);
1018         }
1019       }
1020     }
1021 
1022     @Override
1023     protected int getReportingPeriod() {
1024       int period = this.perClientRunRows / 100;
1025       return period == 0 ? this.perClientRunRows : period;
1026     }
1027 
1028     @Override
1029     protected void testTakedown() throws IOException {
1030       super.testTakedown();
1031       if (this.reportLatency) {
1032         LOG.info("randomRead latency log (ms): " + Arrays.toString(times));
1033       }
1034     }
1035   }
1036 
1037   static class RandomWriteTest extends Test {
1038     RandomWriteTest(Configuration conf, TestOptions options, Status status) {
1039       super(conf, options, status);
1040     }
1041 
1042     @Override
1043     void testRow(final int i) throws IOException {
1044       byte [] row = getRandomRow(this.rand, this.totalRows);
1045       Put put = new Put(row);
1046       byte[] value = generateValue(this.rand);
1047       put.add(FAMILY_NAME, QUALIFIER_NAME, value);
1048       put.setWriteToWAL(writeToWAL);
1049       table.put(put);
1050     }
1051   }
1052 
1053   static class ScanTest extends Test {
1054     private ResultScanner testScanner;
1055 
1056     ScanTest(Configuration conf, TestOptions options, Status status) {
1057       super(conf, options, status);
1058     }
1059 
1060     @Override
1061     void testSetup() throws IOException {
1062       super.testSetup();
1063     }
1064 
1065     @Override
1066     void testTakedown() throws IOException {
1067       if (this.testScanner != null) {
1068         this.testScanner.close();
1069       }
1070       super.testTakedown();
1071     }
1072 
1073 
1074     @Override
1075     void testRow(final int i) throws IOException {
1076       if (this.testScanner == null) {
1077         Scan scan = new Scan(format(this.startRow));
1078         scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
1079         this.testScanner = table.getScanner(scan);
1080       }
1081       testScanner.next();
1082     }
1083 
1084   }
1085 
1086   static class SequentialReadTest extends Test {
1087     SequentialReadTest(Configuration conf, TestOptions options, Status status) {
1088       super(conf, options, status);
1089     }
1090 
1091     @Override
1092     void testRow(final int i) throws IOException {
1093       Get get = new Get(format(i));
1094       get.addColumn(FAMILY_NAME, QUALIFIER_NAME);
1095       table.get(get);
1096     }
1097   }
1098 
1099   static class SequentialWriteTest extends Test {
1100     SequentialWriteTest(Configuration conf, TestOptions options, Status status) {
1101       super(conf, options, status);
1102     }
1103 
1104     @Override
1105     void testRow(final int i) throws IOException {
1106       Put put = new Put(format(i));
1107       byte[] value = generateValue(this.rand);
1108       put.add(FAMILY_NAME, QUALIFIER_NAME, value);
1109       put.setWriteToWAL(writeToWAL);
1110       table.put(put);
1111     }
1112   }
1113 
1114   static class FilteredScanTest extends Test {
1115     protected static final Log LOG = LogFactory.getLog(FilteredScanTest.class.getName());
1116 
1117     FilteredScanTest(Configuration conf, TestOptions options, Status status) {
1118       super(conf, options, status);
1119     }
1120 
1121     @Override
1122     void testRow(int i) throws IOException {
1123       byte[] value = generateValue(this.rand);
1124       Scan scan = constructScan(value);
1125       ResultScanner scanner = null;
1126       try {
1127         scanner = this.table.getScanner(scan);
1128         while (scanner.next() != null) {
1129         }
1130       } finally {
1131         if (scanner != null) scanner.close();
1132       }
1133     }
1134 
1135     protected Scan constructScan(byte[] valuePrefix) throws IOException {
1136       Filter filter = new SingleColumnValueFilter(
1137           FAMILY_NAME, QUALIFIER_NAME, CompareFilter.CompareOp.EQUAL,
1138           new BinaryComparator(valuePrefix)
1139       );
1140       Scan scan = new Scan();
1141       scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
1142       scan.setFilter(filter);
1143       return scan;
1144     }
1145   }
1146 
1147   /**
1148    * Compute a throughput rate in MB/s.
1149    * @param rows Number of records consumed.
1150    * @param timeMs Time taken in milliseconds.
1151    * @return String value with label, ie '123.76 MB/s'
1152    */
1153   private static String calculateMbps(int rows, long timeMs) {
1154     // MB/s = ((totalRows * ROW_SIZE_BYTES) / totalTimeMS)
1155     //        * 1000 MS_PER_SEC / (1024 * 1024) BYTES_PER_MB
1156     BigDecimal rowSize =
1157       BigDecimal.valueOf(VALUE_LENGTH + VALUE_LENGTH + FAMILY_NAME.length + QUALIFIER_NAME.length);
1158     BigDecimal mbps = BigDecimal.valueOf(rows).multiply(rowSize, CXT)
1159       .divide(BigDecimal.valueOf(timeMs), CXT).multiply(MS_PER_SEC, CXT)
1160       .divide(BYTES_PER_MB, CXT);
1161     return FMT.format(mbps) + " MB/s";
1162   }
1163 
1164   /*
1165    * Format passed integer.
1166    * @param number
1167    * @return Returns zero-prefixed ROW_LENGTH-byte wide decimal version of passed
1168    * number (Does absolute in case number is negative).
1169    */
1170   public static byte [] format(final int number) {
1171     byte [] b = new byte[ROW_LENGTH];
1172     int d = Math.abs(number);
1173     for (int i = b.length - 1; i >= 0; i--) {
1174       b[i] = (byte)((d % 10) + '0');
1175       d /= 10;
1176     }
1177     return b;
1178   }
1179 
1180   /*
1181    * This method takes some time and is done inline uploading data.  For
1182    * example, doing the mapfile test, generation of the key and value
1183    * consumes about 30% of CPU time.
1184    * @return Generated random value to insert into a table cell.
1185    */
1186   public static byte[] generateValue(final Random r) {
1187     byte [] b = new byte [VALUE_LENGTH];
1188     r.nextBytes(b);
1189     return b;
1190   }
1191 
1192   static byte [] getRandomRow(final Random random, final int totalRows) {
1193     return format(random.nextInt(Integer.MAX_VALUE) % totalRows);
1194   }
1195 
1196   long runOneClient(final Class<? extends Test> cmd, final int startRow,
1197       final int perClientRunRows, final int totalRows, final float sampleRate,
1198       boolean flushCommits, boolean writeToWAL, boolean reportLatency,
1199       final Status status)
1200   throws IOException {
1201     status.setStatus("Start " + cmd + " at offset " + startRow + " for " +
1202       perClientRunRows + " rows");
1203     long totalElapsedTime = 0;
1204 
1205     Test t = null;
1206     TestOptions options = new TestOptions(startRow, perClientRunRows, totalRows,
1207       sampleRate, getTableDescriptor().getName(), flushCommits, writeToWAL, reportLatency);
1208     try {
1209       Constructor<? extends Test> constructor = cmd.getDeclaredConstructor(
1210           Configuration.class, TestOptions.class, Status.class);
1211       t = constructor.newInstance(this.conf, options, status);
1212     } catch (NoSuchMethodException e) {
1213       throw new IllegalArgumentException("Invalid command class: " +
1214           cmd.getName() + ".  It does not provide a constructor as described by" +
1215           "the javadoc comment.  Available constructors are: " +
1216           Arrays.toString(cmd.getConstructors()));
1217     } catch (Exception e) {
1218       throw new IllegalStateException("Failed to construct command class", e);
1219     }
1220     totalElapsedTime = t.test();
1221 
1222     status.setStatus("Finished " + cmd + " in " + totalElapsedTime +
1223       "ms at offset " + startRow + " for " + perClientRunRows + " rows" +
1224       " (" + calculateMbps((int)(perClientRunRows * sampleRate), totalElapsedTime) + ")");
1225     return totalElapsedTime;
1226   }
1227 
1228   private void runNIsOne(final Class<? extends Test> cmd) throws IOException {
1229     Status status = new Status() {
1230       public void setStatus(String msg) throws IOException {
1231         LOG.info(msg);
1232       }
1233     };
1234 
1235     HBaseAdmin admin = null;
1236     try {
1237       admin = new HBaseAdmin(this.conf);
1238       checkTable(admin);
1239       runOneClient(cmd, 0, this.R, this.R, this.sampleRate, this.flushCommits,
1240         this.writeToWAL, this.writeToWAL, status);
1241     } catch (Exception e) {
1242       LOG.error("Failed", e);
1243     } finally {
1244       if (admin != null) admin.close();
1245     }
1246   }
1247 
1248   private void runTest(final Class<? extends Test> cmd) throws IOException,
1249           InterruptedException, ClassNotFoundException {
1250     MiniHBaseCluster hbaseMiniCluster = null;
1251     MiniDFSCluster dfsCluster = null;
1252     MiniZooKeeperCluster zooKeeperCluster = null;
1253     if (this.miniCluster) {
1254       dfsCluster = new MiniDFSCluster(conf, 2, true, (String[])null);
1255       zooKeeperCluster = new MiniZooKeeperCluster();
1256       int zooKeeperPort = zooKeeperCluster.startup(new File(System.getProperty("java.io.tmpdir")));
1257 
1258       // mangle the conf so that the fs parameter points to the minidfs we
1259       // just started up
1260       FileSystem fs = dfsCluster.getFileSystem();
1261       conf.set("fs.default.name", fs.getUri().toString());
1262       conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, Integer.toString(zooKeeperPort));
1263       Path parentdir = fs.getHomeDirectory();
1264       conf.set(HConstants.HBASE_DIR, parentdir.toString());
1265       fs.mkdirs(parentdir);
1266       FSUtils.setVersion(fs, parentdir);
1267       hbaseMiniCluster = new MiniHBaseCluster(this.conf, N);
1268     }
1269 
1270     try {
1271       if (N == 1) {
1272         // If there is only one client and one HRegionServer, we assume nothing
1273         // has been set up at all.
1274         runNIsOne(cmd);
1275       } else {
1276         // Else, run
1277         runNIsMoreThanOne(cmd);
1278       }
1279     } finally {
1280       if(this.miniCluster) {
1281         if (hbaseMiniCluster != null) hbaseMiniCluster.shutdown();
1282         if (zooKeeperCluster != null) zooKeeperCluster.shutdown();
1283         HBaseTestCase.shutdownDfs(dfsCluster);
1284       }
1285     }
1286   }
1287 
1288   protected void printUsage() {
1289     printUsage(null);
1290   }
1291 
1292   protected void printUsage(final String message) {
1293     if (message != null && message.length() > 0) {
1294       System.err.println(message);
1295     }
1296     System.err.println("Usage: java " + this.getClass().getName() + " \\");
1297     System.err.println("  [--miniCluster] [--nomapred] [--rows=ROWS] <command> <nclients>");
1298     System.err.println();
1299     System.err.println("Options:");
1300     System.err.println(" miniCluster     Run the test on an HBaseMiniCluster");
1301     System.err.println(" nomapred        Run multiple clients using threads " +
1302       "(rather than use mapreduce)");
1303     System.err.println(" rows            Rows each client runs. Default: One million");
1304     System.err.println(" sampleRate      Execute test on a sample of total " +
1305       "rows. Only supported by randomRead. Default: 1.0");
1306     System.err.println(" flushCommits    Used to determine if the test should flush the table. " +
1307       "Default: false");
1308     System.err.println(" writeToWAL      Set writeToWAL on puts. Default: True");
1309     System.err.println(" presplit        Create presplit table. Recommended for accurate perf " +
1310       "analysis (see guide).  Default: disabled");
1311     System.err.println(" latency         Set to report operation latencies. " +
1312       "Currently only supported by randomRead test. Default: False");
1313     System.err.println();
1314     System.err.println("Command:");
1315     for (CmdDescriptor command : commands.values()) {
1316       System.err.println(String.format(" %-15s %s", command.getName(), command.getDescription()));
1317     }
1318     System.err.println();
1319     System.err.println("Args:");
1320     System.err.println(" nclients        Integer. Required. Total number of " +
1321       "clients (and HRegionServers)");
1322     System.err.println("                 running: 1 <= value <= 500");
1323     System.err.println("Examples:");
1324     System.err.println(" To run a single evaluation client:");
1325     System.err.println(" $ bin/hbase " + this.getClass().getName()
1326         + " sequentialWrite 1");
1327   }
1328 
1329   private void getArgs(final int start, final String[] args) {
1330     if(start + 1 > args.length) {
1331       throw new IllegalArgumentException("must supply the number of clients");
1332     }
1333     N = Integer.parseInt(args[start]);
1334     if (N < 1) {
1335       throw new IllegalArgumentException("Number of clients must be > 1");
1336     }
1337     // Set total number of rows to write.
1338     this.R = this.R * N;
1339   }
1340 
1341   public int doCommandLine(final String[] args) {
1342     // Process command-line args. TODO: Better cmd-line processing
1343     // (but hopefully something not as painful as cli options).
1344     int errCode = -1;
1345     if (args.length < 1) {
1346       printUsage();
1347       return errCode;
1348     }
1349 
1350     try {
1351       for (int i = 0; i < args.length; i++) {
1352         String cmd = args[i];
1353         if (cmd.equals("-h") || cmd.startsWith("--h")) {
1354           printUsage();
1355           errCode = 0;
1356           break;
1357         }
1358 
1359         final String miniClusterArgKey = "--miniCluster";
1360         if (cmd.startsWith(miniClusterArgKey)) {
1361           this.miniCluster = true;
1362           continue;
1363         }
1364 
1365         final String nmr = "--nomapred";
1366         if (cmd.startsWith(nmr)) {
1367           this.nomapred = true;
1368           continue;
1369         }
1370 
1371         final String rows = "--rows=";
1372         if (cmd.startsWith(rows)) {
1373           this.R = Integer.parseInt(cmd.substring(rows.length()));
1374           continue;
1375         }
1376 
1377         final String sampleRate = "--sampleRate=";
1378         if (cmd.startsWith(sampleRate)) {
1379           this.sampleRate = Float.parseFloat(cmd.substring(sampleRate.length()));
1380           continue;
1381         }
1382 
1383         final String flushCommits = "--flushCommits=";
1384         if (cmd.startsWith(flushCommits)) {
1385           this.flushCommits = Boolean.parseBoolean(cmd.substring(flushCommits.length()));
1386           continue;
1387         }
1388 
1389         final String writeToWAL = "--writeToWAL=";
1390         if (cmd.startsWith(writeToWAL)) {
1391           this.writeToWAL = Boolean.parseBoolean(cmd.substring(writeToWAL.length()));
1392           continue;
1393         }
1394 
1395         final String presplit = "--presplit=";
1396         if (cmd.startsWith(presplit)) {
1397           this.presplitRegions = Integer.parseInt(cmd.substring(presplit.length()));
1398           continue;
1399         }
1400 
1401         final String latency = "--latency";
1402         if (cmd.startsWith(latency)) {
1403           this.reportLatency = true;
1404           continue;
1405         }
1406 
1407         Class<? extends Test> cmdClass = determineCommandClass(cmd);
1408         if (cmdClass != null) {
1409           getArgs(i + 1, args);
1410           runTest(cmdClass);
1411           errCode = 0;
1412           break;
1413         }
1414 
1415         printUsage();
1416         break;
1417       }
1418     } catch (Exception e) {
1419       e.printStackTrace();
1420     }
1421 
1422     return errCode;
1423   }
1424 
1425   private Class<? extends Test> determineCommandClass(String cmd) {
1426     CmdDescriptor descriptor = commands.get(cmd);
1427     return descriptor != null ? descriptor.getCmdClass() : null;
1428   }
1429 
1430   public static void main(final String[] args) {
1431     Configuration c = HBaseConfiguration.create();
1432     System.exit(new PerformanceEvaluation(c).doCommandLine(args));
1433   }
1434 }