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