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