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