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