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