View Javadoc

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