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