View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.mapreduce;
19  
20  import java.io.IOException;
21  import java.io.InputStreamReader;
22  import java.io.OutputStreamWriter;
23  import java.security.MessageDigest;
24  import java.security.NoSuchAlgorithmException;
25  import java.util.ArrayList;
26  import java.util.Collections;
27  import java.util.List;
28  import java.util.Properties;
29  
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.conf.Configured;
34  import org.apache.hadoop.fs.FSDataInputStream;
35  import org.apache.hadoop.fs.FileSystem;
36  import org.apache.hadoop.fs.Path;
37  import org.apache.hadoop.hbase.Cell;
38  import org.apache.hadoop.hbase.HBaseConfiguration;
39  import org.apache.hadoop.hbase.HConstants;
40  import org.apache.hadoop.hbase.TableName;
41  import org.apache.hadoop.hbase.client.Connection;
42  import org.apache.hadoop.hbase.client.ConnectionFactory;
43  import org.apache.hadoop.hbase.client.Result;
44  import org.apache.hadoop.hbase.client.Scan;
45  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
46  import org.apache.hadoop.hbase.util.Bytes;
47  import org.apache.hadoop.hbase.util.Pair;
48  import org.apache.hadoop.io.MapFile;
49  import org.apache.hadoop.io.NullWritable;
50  import org.apache.hadoop.io.SequenceFile;
51  import org.apache.hadoop.mapreduce.Job;
52  import org.apache.hadoop.mapreduce.Reducer;
53  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
54  import org.apache.hadoop.mapreduce.lib.output.MapFileOutputFormat;
55  import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
56  import org.apache.hadoop.util.GenericOptionsParser;
57  import org.apache.hadoop.util.Tool;
58  import org.apache.hadoop.util.ToolRunner;
59  
60  import com.google.common.base.Charsets;
61  import com.google.common.base.Throwables;
62  import com.google.common.collect.Ordering;
63  
64  public class HashTable extends Configured implements Tool {
65  
66    private static final Log LOG = LogFactory.getLog(HashTable.class);
67    
68    private static final int DEFAULT_BATCH_SIZE = 8000;
69    
70    private final static String HASH_BATCH_SIZE_CONF_KEY = "hash.batch.size";
71    final static String PARTITIONS_FILE_NAME = "partitions";
72    final static String MANIFEST_FILE_NAME = "manifest";
73    final static String HASH_DATA_DIR = "hashes";
74    final static String OUTPUT_DATA_FILE_PREFIX = "part-r-";
75    final static String IGNORE_TIMESTAMPS = "ignoreTimestamps";
76    private final static String TMP_MANIFEST_FILE_NAME = "manifest.tmp";
77    
78    TableHash tableHash = new TableHash();
79    Path destPath;
80    
81    public HashTable(Configuration conf) {
82      super(conf);
83    }
84    
85    public static class TableHash {
86      
87      Path hashDir;
88      
89      String tableName;
90      String families = null;
91      long batchSize = DEFAULT_BATCH_SIZE;
92      int numHashFiles = 0;
93      byte[] startRow = HConstants.EMPTY_START_ROW;
94      byte[] stopRow = HConstants.EMPTY_END_ROW;
95      int scanBatch = 0;
96      int versions = -1;
97      long startTime = 0;
98      long endTime = 0;
99      boolean ignoreTimestamps;
100     
101     List<ImmutableBytesWritable> partitions;
102     
103     public static TableHash read(Configuration conf, Path hashDir) throws IOException {
104       TableHash tableHash = new TableHash();
105       FileSystem fs = hashDir.getFileSystem(conf);
106       tableHash.hashDir = hashDir;
107       tableHash.readPropertiesFile(fs, new Path(hashDir, MANIFEST_FILE_NAME));
108       tableHash.readPartitionFile(fs, conf, new Path(hashDir, PARTITIONS_FILE_NAME));
109       return tableHash;
110     }
111     
112     void writePropertiesFile(FileSystem fs, Path path) throws IOException {
113       Properties p = new Properties();
114       p.setProperty("table", tableName);
115       if (families != null) {
116         p.setProperty("columnFamilies", families);
117       }
118       p.setProperty("targetBatchSize", Long.toString(batchSize));
119       p.setProperty("numHashFiles", Integer.toString(numHashFiles));
120       if (!isTableStartRow(startRow)) {
121         p.setProperty("startRowHex", Bytes.toHex(startRow));
122       }
123       if (!isTableEndRow(stopRow)) {
124         p.setProperty("stopRowHex", Bytes.toHex(stopRow));
125       }
126       if (scanBatch > 0) {
127         p.setProperty("scanBatch", Integer.toString(scanBatch));
128       }
129       if (versions >= 0) {
130         p.setProperty("versions", Integer.toString(versions));
131       }
132       if (startTime != 0) {
133         p.setProperty("startTimestamp", Long.toString(startTime));
134       }
135       if (endTime != 0) {
136         p.setProperty("endTimestamp", Long.toString(endTime));
137       }
138       
139       try (OutputStreamWriter osw = new OutputStreamWriter(fs.create(path), Charsets.UTF_8)) {
140         p.store(osw, null);
141       }
142     }
143 
144     void readPropertiesFile(FileSystem fs, Path path) throws IOException {
145       Properties p = new Properties();
146       try (FSDataInputStream in = fs.open(path)) {
147         try (InputStreamReader isr = new InputStreamReader(in, Charsets.UTF_8)) {
148           p.load(isr);
149         }
150       }
151       tableName = p.getProperty("table");
152       families = p.getProperty("columnFamilies");
153       batchSize = Long.parseLong(p.getProperty("targetBatchSize"));
154       numHashFiles = Integer.parseInt(p.getProperty("numHashFiles"));
155       
156       String startRowHex = p.getProperty("startRowHex");
157       if (startRowHex != null) {
158         startRow = Bytes.fromHex(startRowHex);
159       }
160       String stopRowHex = p.getProperty("stopRowHex");
161       if (stopRowHex != null) {
162         stopRow = Bytes.fromHex(stopRowHex);
163       }
164       
165       String scanBatchString = p.getProperty("scanBatch");
166       if (scanBatchString != null) {
167         scanBatch = Integer.parseInt(scanBatchString);
168       }
169       
170       String versionString = p.getProperty("versions");
171       if (versionString != null) {
172         versions = Integer.parseInt(versionString);
173       }
174       
175       String startTimeString = p.getProperty("startTimestamp");
176       if (startTimeString != null) {
177         startTime = Long.parseLong(startTimeString);
178       }
179       
180       String endTimeString = p.getProperty("endTimestamp");
181       if (endTimeString != null) {
182         endTime = Long.parseLong(endTimeString);
183       }
184     }
185     
186     Scan initScan() throws IOException {
187       Scan scan = new Scan();
188       scan.setCacheBlocks(false);
189       if (startTime != 0 || endTime != 0) {
190         scan.setTimeRange(startTime, endTime == 0 ? HConstants.LATEST_TIMESTAMP : endTime);
191       }
192       if (scanBatch > 0) {
193         scan.setBatch(scanBatch);
194       }
195       if (versions >= 0) {
196         scan.setMaxVersions(versions);
197       }
198       if (!isTableStartRow(startRow)) {
199         scan.setStartRow(startRow);
200       }
201       if (!isTableEndRow(stopRow)) {
202         scan.setStopRow(stopRow);
203       }
204       if(families != null) {
205         for(String fam : families.split(",")) {
206           scan.addFamily(Bytes.toBytes(fam));
207         }
208       }
209       return scan;
210     }
211     
212     /**
213      * Choose partitions between row ranges to hash to a single output file
214      * Selects region boundaries that fall within the scan range, and groups them
215      * into the desired number of partitions.
216      */
217     void selectPartitions(Pair<byte[][], byte[][]> regionStartEndKeys) {
218       List<byte[]> startKeys = new ArrayList<byte[]>();
219       for (int i = 0; i < regionStartEndKeys.getFirst().length; i++) {
220         byte[] regionStartKey = regionStartEndKeys.getFirst()[i];
221         byte[] regionEndKey = regionStartEndKeys.getSecond()[i];
222         
223         // if scan begins after this region, or starts before this region, then drop this region
224         // in other words:
225         //   IF (scan begins before the end of this region
226         //      AND scan ends before the start of this region)
227         //   THEN include this region
228         if ((isTableStartRow(startRow) || isTableEndRow(regionEndKey)
229             || Bytes.compareTo(startRow, regionEndKey) < 0)
230           && (isTableEndRow(stopRow) || isTableStartRow(regionStartKey)
231             || Bytes.compareTo(stopRow, regionStartKey) > 0)) {
232           startKeys.add(regionStartKey);
233         }
234       }
235       
236       int numRegions = startKeys.size();
237       if (numHashFiles == 0) {
238         numHashFiles = numRegions / 100;
239       }
240       if (numHashFiles == 0) {
241         numHashFiles = 1;
242       }
243       if (numHashFiles > numRegions) {
244         // can't partition within regions
245         numHashFiles = numRegions;
246       }
247       
248       // choose a subset of start keys to group regions into ranges
249       partitions = new ArrayList<ImmutableBytesWritable>(numHashFiles - 1);
250       // skip the first start key as it is not a partition between ranges.
251       for (long i = 1; i < numHashFiles; i++) {
252         int splitIndex = (int) (numRegions * i / numHashFiles);
253         partitions.add(new ImmutableBytesWritable(startKeys.get(splitIndex)));
254       }
255     }
256     
257     void writePartitionFile(Configuration conf, Path path) throws IOException {
258       FileSystem fs = path.getFileSystem(conf);
259       @SuppressWarnings("deprecation")
260       SequenceFile.Writer writer = SequenceFile.createWriter(
261         fs, conf, path, ImmutableBytesWritable.class, NullWritable.class);
262       
263       for (int i = 0; i < partitions.size(); i++) {
264         writer.append(partitions.get(i), NullWritable.get());
265       }
266       writer.close();
267     }
268     
269     private void readPartitionFile(FileSystem fs, Configuration conf, Path path)
270          throws IOException {
271       @SuppressWarnings("deprecation")
272       SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
273       ImmutableBytesWritable key = new ImmutableBytesWritable();
274       partitions = new ArrayList<ImmutableBytesWritable>();
275       while (reader.next(key)) {
276         partitions.add(new ImmutableBytesWritable(key.copyBytes()));
277       }
278       reader.close();
279       
280       if (!Ordering.natural().isOrdered(partitions)) {
281         throw new IOException("Partitions are not ordered!");
282       }
283     }
284 
285     @Override
286     public String toString() {
287       StringBuilder sb = new StringBuilder();
288       sb.append("tableName=").append(tableName);
289       if (families != null) {
290         sb.append(", families=").append(families);
291       }
292       sb.append(", batchSize=").append(batchSize);
293       sb.append(", numHashFiles=").append(numHashFiles);
294       if (!isTableStartRow(startRow)) {
295         sb.append(", startRowHex=").append(Bytes.toHex(startRow));
296       }
297       if (!isTableEndRow(stopRow)) {
298         sb.append(", stopRowHex=").append(Bytes.toHex(stopRow));
299       }
300       if (scanBatch >= 0) {
301         sb.append(", scanBatch=").append(scanBatch);
302       }
303       if (versions >= 0) {
304         sb.append(", versions=").append(versions);
305       }
306       if (startTime != 0) {
307         sb.append("startTime=").append(startTime);
308       }
309       if (endTime != 0) {
310         sb.append("endTime=").append(endTime);
311       }
312       return sb.toString();
313     }
314     
315     static String getDataFileName(int hashFileIndex) {
316       return String.format(HashTable.OUTPUT_DATA_FILE_PREFIX + "%05d", hashFileIndex);
317     }
318     
319     /**
320      * Open a TableHash.Reader starting at the first hash at or after the given key.
321      * @throws IOException 
322      */
323     public Reader newReader(Configuration conf, ImmutableBytesWritable startKey)
324         throws IOException {
325       return new Reader(conf, startKey);
326     }
327     
328     public class Reader implements java.io.Closeable {
329       private final Configuration conf;
330       
331       private int hashFileIndex;
332       private MapFile.Reader mapFileReader;
333       
334       private boolean cachedNext;
335       private ImmutableBytesWritable key;
336       private ImmutableBytesWritable hash;
337       
338       Reader(Configuration conf, ImmutableBytesWritable startKey) throws IOException {
339         this.conf = conf;
340         int partitionIndex = Collections.binarySearch(partitions, startKey);
341         if (partitionIndex >= 0) {
342           // if the key is equal to a partition, then go the file after that partition
343           hashFileIndex = partitionIndex+1;
344         } else {
345           // if the key is between partitions, then go to the file between those partitions
346           hashFileIndex = -1-partitionIndex;
347         }
348         openHashFile();
349         
350         // MapFile's don't make it easy to seek() so that the subsequent next() returns
351         // the desired key/value pair.  So we cache it for the first call of next().
352         hash = new ImmutableBytesWritable();
353         key = (ImmutableBytesWritable) mapFileReader.getClosest(startKey, hash);
354         if (key == null) {
355           cachedNext = false;
356           hash = null;
357         } else {
358           cachedNext = true;
359         }
360       }
361       
362       /**
363        * Read the next key/hash pair.
364        * Returns true if such a pair exists and false when at the end of the data.
365        */
366       public boolean next() throws IOException {
367         if (cachedNext) {
368           cachedNext = false;
369           return true;
370         }
371         key = new ImmutableBytesWritable();
372         hash = new ImmutableBytesWritable();
373         while (true) {
374           boolean hasNext = mapFileReader.next(key, hash);
375           if (hasNext) {
376             return true;
377           }
378           hashFileIndex++;
379           if (hashFileIndex < TableHash.this.numHashFiles) {
380             mapFileReader.close();
381             openHashFile();
382           } else {
383             key = null;
384             hash = null;
385             return false;
386           }
387         }
388       }
389       
390       /**
391        * Get the current key
392        * @return the current key or null if there is no current key
393        */
394       public ImmutableBytesWritable getCurrentKey() {
395         return key;
396       }
397       
398       /**
399        * Get the current hash
400        * @return the current hash or null if there is no current hash
401        */
402       public ImmutableBytesWritable getCurrentHash() {
403         return hash;
404       }
405       
406       private void openHashFile() throws IOException {
407         if (mapFileReader != null) {
408           mapFileReader.close();
409         }
410         Path dataDir = new Path(TableHash.this.hashDir, HASH_DATA_DIR);
411         Path dataFile = new Path(dataDir, getDataFileName(hashFileIndex));
412         mapFileReader = new MapFile.Reader(dataFile, conf);
413       }
414 
415       @Override
416       public void close() throws IOException {
417         mapFileReader.close();
418       }
419     }
420   }
421   
422   static boolean isTableStartRow(byte[] row) {
423     return Bytes.equals(HConstants.EMPTY_START_ROW, row);
424   }
425   
426   static boolean isTableEndRow(byte[] row) {
427     return Bytes.equals(HConstants.EMPTY_END_ROW, row);
428   }
429   
430   public Job createSubmittableJob(String[] args) throws IOException {
431     Path partitionsPath = new Path(destPath, PARTITIONS_FILE_NAME);
432     generatePartitions(partitionsPath);
433     
434     Job job = Job.getInstance(getConf(),
435           getConf().get("mapreduce.job.name", "hashTable_" + tableHash.tableName));
436     Configuration jobConf = job.getConfiguration();
437     jobConf.setLong(HASH_BATCH_SIZE_CONF_KEY, tableHash.batchSize);
438     jobConf.setBoolean(IGNORE_TIMESTAMPS, tableHash.ignoreTimestamps);
439     job.setJarByClass(HashTable.class);
440 
441     TableMapReduceUtil.initTableMapperJob(tableHash.tableName, tableHash.initScan(),
442         HashMapper.class, ImmutableBytesWritable.class, ImmutableBytesWritable.class, job);
443     
444     // use a TotalOrderPartitioner and reducers to group region output into hash files
445     job.setPartitionerClass(TotalOrderPartitioner.class);
446     TotalOrderPartitioner.setPartitionFile(jobConf, partitionsPath);
447     job.setReducerClass(Reducer.class);  // identity reducer
448     job.setNumReduceTasks(tableHash.numHashFiles);
449     job.setOutputKeyClass(ImmutableBytesWritable.class);
450     job.setOutputValueClass(ImmutableBytesWritable.class);
451     job.setOutputFormatClass(MapFileOutputFormat.class);
452     FileOutputFormat.setOutputPath(job, new Path(destPath, HASH_DATA_DIR));
453     
454     return job;
455   }
456   
457   private void generatePartitions(Path partitionsPath) throws IOException {
458     Connection connection = ConnectionFactory.createConnection(getConf());
459     Pair<byte[][], byte[][]> regionKeys
460       = connection.getRegionLocator(TableName.valueOf(tableHash.tableName)).getStartEndKeys();
461     connection.close();
462     
463     tableHash.selectPartitions(regionKeys);
464     LOG.info("Writing " + tableHash.partitions.size() + " partition keys to " + partitionsPath);
465     
466     tableHash.writePartitionFile(getConf(), partitionsPath);
467   }
468   
469   static class ResultHasher {
470     private MessageDigest digest;
471     
472     private boolean batchStarted = false;
473     private ImmutableBytesWritable batchStartKey;
474     private ImmutableBytesWritable batchHash;
475     private long batchSize = 0;
476     boolean ignoreTimestamps;
477     
478     
479     public ResultHasher() {
480       try {
481         digest = MessageDigest.getInstance("MD5");
482       } catch (NoSuchAlgorithmException e) {
483         Throwables.propagate(e);
484       }
485     }
486     
487     public void startBatch(ImmutableBytesWritable row) {
488       if (batchStarted) {
489         throw new RuntimeException("Cannot start new batch without finishing existing one.");
490       }
491       batchStarted = true;
492       batchSize = 0;
493       batchStartKey = row;
494       batchHash = null;
495     }
496     
497     public void hashResult(Result result) {
498       if (!batchStarted) {
499         throw new RuntimeException("Cannot add to batch that has not been started.");
500       }
501       for (Cell cell : result.rawCells()) {
502         int rowLength = cell.getRowLength();
503         int familyLength = cell.getFamilyLength();
504         int qualifierLength = cell.getQualifierLength();
505         int valueLength = cell.getValueLength();
506         digest.update(cell.getRowArray(), cell.getRowOffset(), rowLength);
507         digest.update(cell.getFamilyArray(), cell.getFamilyOffset(), familyLength);
508         digest.update(cell.getQualifierArray(), cell.getQualifierOffset(), qualifierLength);
509         long ts = cell.getTimestamp();
510         if(!ignoreTimestamps) {
511           for (int i = 8; i > 0; i--) {
512             digest.update((byte) ts);
513             ts >>>= 8;
514           }
515         }
516         digest.update(cell.getValueArray(), cell.getValueOffset(), valueLength);
517         
518         batchSize += rowLength + familyLength + qualifierLength + 8 + valueLength;
519       }
520     }
521     
522     public void finishBatch() {
523       if (!batchStarted) {
524         throw new RuntimeException("Cannot finish batch that has not started.");
525       }
526       batchStarted = false;
527       batchHash = new ImmutableBytesWritable(digest.digest());
528     }
529 
530     public boolean isBatchStarted() {
531       return batchStarted;
532     }
533 
534     public ImmutableBytesWritable getBatchStartKey() {
535       return batchStartKey;
536     }
537 
538     public ImmutableBytesWritable getBatchHash() {
539       return batchHash;
540     }
541 
542     public long getBatchSize() {
543       return batchSize;
544     }
545   }
546   
547   public static class HashMapper
548     extends TableMapper<ImmutableBytesWritable, ImmutableBytesWritable> {
549     
550     private ResultHasher hasher;
551     private long targetBatchSize;
552     
553     private ImmutableBytesWritable currentRow;
554     
555     @Override
556     protected void setup(Context context) throws IOException, InterruptedException {
557       targetBatchSize = context.getConfiguration()
558           .getLong(HASH_BATCH_SIZE_CONF_KEY, DEFAULT_BATCH_SIZE);
559       hasher = new ResultHasher();
560       hasher.ignoreTimestamps = context.getConfiguration().
561         getBoolean(IGNORE_TIMESTAMPS, false);
562       TableSplit split = (TableSplit) context.getInputSplit();
563       hasher.startBatch(new ImmutableBytesWritable(split.getStartRow()));
564     }
565     
566     @Override
567     protected void map(ImmutableBytesWritable key, Result value, Context context)
568         throws IOException, InterruptedException {
569       
570       if (currentRow == null || !currentRow.equals(key)) {
571         currentRow = new ImmutableBytesWritable(key); // not immutable
572         
573         if (hasher.getBatchSize() >= targetBatchSize) {
574           hasher.finishBatch();
575           context.write(hasher.getBatchStartKey(), hasher.getBatchHash());
576           hasher.startBatch(currentRow);
577         }
578       }
579       
580       hasher.hashResult(value);
581     }
582 
583     @Override
584     protected void cleanup(Context context) throws IOException, InterruptedException {
585       hasher.finishBatch();
586       context.write(hasher.getBatchStartKey(), hasher.getBatchHash());
587     }
588   }
589   
590   private void writeTempManifestFile() throws IOException {
591     Path tempManifestPath = new Path(destPath, TMP_MANIFEST_FILE_NAME);
592     FileSystem fs = tempManifestPath.getFileSystem(getConf());
593     tableHash.writePropertiesFile(fs, tempManifestPath);
594   }
595   
596   private void completeManifest() throws IOException {
597     Path tempManifestPath = new Path(destPath, TMP_MANIFEST_FILE_NAME);
598     Path manifestPath = new Path(destPath, MANIFEST_FILE_NAME);
599     FileSystem fs = tempManifestPath.getFileSystem(getConf());
600     fs.rename(tempManifestPath, manifestPath);
601   }
602   
603   private static final int NUM_ARGS = 2;
604   private static void printUsage(final String errorMsg) {
605     if (errorMsg != null && errorMsg.length() > 0) {
606       System.err.println("ERROR: " + errorMsg);
607       System.err.println();
608     }
609     System.err.println("Usage: HashTable [options] <tablename> <outputpath>");
610     System.err.println();
611     System.err.println("Options:");
612     System.err.println(" batchsize         the target amount of bytes to hash in each batch");
613     System.err.println("                   rows are added to the batch until this size is reached");
614     System.err.println("                   (defaults to " + DEFAULT_BATCH_SIZE + " bytes)");
615     System.err.println(" numhashfiles      the number of hash files to create");
616     System.err.println("                   if set to fewer than number of regions then");
617     System.err.println("                   the job will create this number of reducers");
618     System.err.println("                   (defaults to 1/100 of regions -- at least 1)");
619     System.err.println(" startrow          the start row");
620     System.err.println(" stoprow           the stop row");
621     System.err.println(" starttime         beginning of the time range (unixtime in millis)");
622     System.err.println("                   without endtime means from starttime to forever");
623     System.err.println(" endtime           end of the time range.");
624     System.err.println("                   Ignored if no starttime specified.");
625     System.err.println(" scanbatch         scanner batch size to support intra row scans");
626     System.err.println(" versions          number of cell versions to include");
627     System.err.println(" families          comma-separated list of families to include");
628     System.err.println(" ignoreTimestamps  if true, ignores cell timestamps");
629     System.err.println("                   when calculating hashes");
630     System.err.println();
631     System.err.println("Args:");
632     System.err.println(" tablename     Name of the table to hash");
633     System.err.println(" outputpath    Filesystem path to put the output data");
634     System.err.println();
635     System.err.println("Examples:");
636     System.err.println(" To hash 'TestTable' in 32kB batches for a 1 hour window into 50 files:");
637     System.err.println(" $ bin/hbase " +
638         "org.apache.hadoop.hbase.mapreduce.HashTable --batchsize=32000 --numhashfiles=50"
639         + " --starttime=1265875194289 --endtime=1265878794289 --families=cf2,cf3"
640         + " TestTable /hashes/testTable");
641   }
642 
643   private boolean doCommandLine(final String[] args) {
644     if (args.length < NUM_ARGS) {
645       printUsage(null);
646       return false;
647     }
648     try {
649       
650       tableHash.tableName = args[args.length-2];
651       destPath = new Path(args[args.length-1]);
652       
653       for (int i = 0; i < args.length - NUM_ARGS; i++) {
654         String cmd = args[i];
655         if (cmd.equals("-h") || cmd.startsWith("--h")) {
656           printUsage(null);
657           return false;
658         }
659         
660         final String batchSizeArgKey = "--batchsize=";
661         if (cmd.startsWith(batchSizeArgKey)) {
662           tableHash.batchSize = Long.parseLong(cmd.substring(batchSizeArgKey.length()));
663           continue;
664         }
665         
666         final String numHashFilesArgKey = "--numhashfiles=";
667         if (cmd.startsWith(numHashFilesArgKey)) {
668           tableHash.numHashFiles = Integer.parseInt(cmd.substring(numHashFilesArgKey.length()));
669           continue;
670         }
671          
672         final String startRowArgKey = "--startrow=";
673         if (cmd.startsWith(startRowArgKey)) {
674           tableHash.startRow = Bytes.fromHex(cmd.substring(startRowArgKey.length()));
675           continue;
676         }
677         
678         final String stopRowArgKey = "--stoprow=";
679         if (cmd.startsWith(stopRowArgKey)) {
680           tableHash.stopRow = Bytes.fromHex(cmd.substring(stopRowArgKey.length()));
681           continue;
682         }
683         
684         final String startTimeArgKey = "--starttime=";
685         if (cmd.startsWith(startTimeArgKey)) {
686           tableHash.startTime = Long.parseLong(cmd.substring(startTimeArgKey.length()));
687           continue;
688         }
689 
690         final String endTimeArgKey = "--endtime=";
691         if (cmd.startsWith(endTimeArgKey)) {
692           tableHash.endTime = Long.parseLong(cmd.substring(endTimeArgKey.length()));
693           continue;
694         }
695 
696         final String scanBatchArgKey = "--scanbatch=";
697         if (cmd.startsWith(scanBatchArgKey)) {
698           tableHash.scanBatch = Integer.parseInt(cmd.substring(scanBatchArgKey.length()));
699           continue;
700         }
701 
702         final String versionsArgKey = "--versions=";
703         if (cmd.startsWith(versionsArgKey)) {
704           tableHash.versions = Integer.parseInt(cmd.substring(versionsArgKey.length()));
705           continue;
706         }
707 
708         final String familiesArgKey = "--families=";
709         if (cmd.startsWith(familiesArgKey)) {
710           tableHash.families = cmd.substring(familiesArgKey.length());
711           continue;
712         }
713 
714         final String ignoreTimestampsKey = "--ignoreTimestamps=";
715         if (cmd.startsWith(ignoreTimestampsKey)) {
716           tableHash.ignoreTimestamps = Boolean.
717             parseBoolean(cmd.substring(ignoreTimestampsKey.length()));
718           continue;
719         }
720 
721         printUsage("Invalid argument '" + cmd + "'");
722         return false;
723       }
724       if ((tableHash.startTime != 0 || tableHash.endTime != 0)
725           && (tableHash.startTime >= tableHash.endTime)) {
726         printUsage("Invalid time range filter: starttime="
727             + tableHash.startTime + " >=  endtime=" + tableHash.endTime);
728         return false;
729       }
730       
731     } catch (Exception e) {
732       e.printStackTrace();
733       printUsage("Can't start because " + e.getMessage());
734       return false;
735     }
736     return true;
737   }
738 
739   /**
740    * Main entry point.
741    */
742   public static void main(String[] args) throws Exception {
743     int ret = ToolRunner.run(new HashTable(HBaseConfiguration.create()), args);
744     System.exit(ret);
745   }
746 
747   @Override
748   public int run(String[] args) throws Exception {
749     String[] otherArgs = new GenericOptionsParser(getConf(), args).getRemainingArgs();
750     if (!doCommandLine(otherArgs)) {
751       return 1;
752     }
753 
754     Job job = createSubmittableJob(otherArgs);
755     writeTempManifestFile();
756     if (!job.waitForCompletion(true)) {
757       LOG.info("Map-reduce job failed!");
758       return 1;
759     }
760     completeManifest();
761     return 0;
762   }
763 
764 }