View Javadoc

1   /**
2    * Copyright 2010 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.mapreduce;
21  
22  import java.io.FileNotFoundException;
23  import java.io.IOException;
24  import java.nio.ByteBuffer;
25  import java.util.ArrayList;
26  import java.util.Arrays;
27  import java.util.Collection;
28  import java.util.Deque;
29  import java.util.HashSet;
30  import java.util.LinkedList;
31  import java.util.List;
32  import java.util.Map;
33  import java.util.Map.Entry;
34  import java.util.Set;
35  import java.util.TreeMap;
36  import java.util.concurrent.Callable;
37  import java.util.concurrent.ExecutionException;
38  import java.util.concurrent.ExecutorService;
39  import java.util.concurrent.Future;
40  import java.util.concurrent.LinkedBlockingQueue;
41  import java.util.concurrent.ThreadPoolExecutor;
42  import java.util.concurrent.TimeUnit;
43  import java.util.concurrent.atomic.AtomicLong;
44  
45  import org.apache.commons.logging.Log;
46  import org.apache.commons.logging.LogFactory;
47  import org.apache.hadoop.conf.Configuration;
48  import org.apache.hadoop.conf.Configured;
49  import org.apache.hadoop.fs.FileStatus;
50  import org.apache.hadoop.fs.FileSystem;
51  import org.apache.hadoop.fs.FileUtil;
52  import org.apache.hadoop.fs.Path;
53  import org.apache.hadoop.hbase.HBaseConfiguration;
54  import org.apache.hadoop.hbase.HColumnDescriptor;
55  import org.apache.hadoop.hbase.HConstants;
56  import org.apache.hadoop.hbase.HTableDescriptor;
57  import org.apache.hadoop.hbase.KeyValue;
58  import org.apache.hadoop.hbase.TableNotFoundException;
59  import org.apache.hadoop.hbase.client.HBaseAdmin;
60  import org.apache.hadoop.hbase.client.HConnection;
61  import org.apache.hadoop.hbase.client.HTable;
62  import org.apache.hadoop.hbase.client.ServerCallable;
63  import org.apache.hadoop.hbase.io.HalfStoreFileReader;
64  import org.apache.hadoop.hbase.io.Reference;
65  import org.apache.hadoop.hbase.io.Reference.Range;
66  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
67  import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
68  import org.apache.hadoop.hbase.io.hfile.HFile;
69  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
70  import org.apache.hadoop.hbase.regionserver.StoreFile;
71  import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
72  import org.apache.hadoop.hbase.util.Bytes;
73  import org.apache.hadoop.hbase.util.Pair;
74  import org.apache.hadoop.util.Tool;
75  import org.apache.hadoop.util.ToolRunner;
76  
77  import com.google.common.collect.HashMultimap;
78  import com.google.common.collect.Multimap;
79  import com.google.common.collect.Multimaps;
80  import com.google.common.util.concurrent.ThreadFactoryBuilder;
81  
82  /**
83   * Tool to load the output of HFileOutputFormat into an existing table.
84   * @see #usage()
85   */
86  public class LoadIncrementalHFiles extends Configured implements Tool {
87  
88    private static Log LOG = LogFactory.getLog(LoadIncrementalHFiles.class);
89    private static final int  TABLE_CREATE_MAX_RETRIES = 20;
90    private static final long TABLE_CREATE_SLEEP = 60000;
91    static AtomicLong regionCount = new AtomicLong(0);
92    private HBaseAdmin hbAdmin;
93    private Configuration cfg;
94  
95    public static String NAME = "completebulkload";
96  
97    public LoadIncrementalHFiles(Configuration conf) throws Exception {
98      super(conf);
99      this.cfg = conf;
100     this.hbAdmin = new HBaseAdmin(conf);
101   }
102 
103   private void usage() {
104     System.err.println("usage: " + NAME +
105         " /path/to/hfileoutputformat-output " +
106         "tablename");
107   }
108 
109   /**
110    * Represents an HFile waiting to be loaded. An queue is used
111    * in this class in order to support the case where a region has
112    * split during the process of the load. When this happens,
113    * the HFile is split into two physical parts across the new
114    * region boundary, and each part is added back into the queue.
115    * The import process finishes when the queue is empty.
116    */
117   static class LoadQueueItem {
118     final byte[] family;
119     final Path hfilePath;
120 
121     public LoadQueueItem(byte[] family, Path hfilePath) {
122       this.family = family;
123       this.hfilePath = hfilePath;
124     }
125 
126     public String toString() {
127       return "family:"+ Bytes.toString(family) + " path:" + hfilePath.toString();
128     }
129   }
130 
131   /**
132    * Walk the given directory for all HFiles, and return a Queue
133    * containing all such files.
134    */
135   private void discoverLoadQueue(Deque<LoadQueueItem> ret, Path hfofDir)
136   throws IOException {
137     FileSystem fs = hfofDir.getFileSystem(getConf());
138 
139     if (!fs.exists(hfofDir)) {
140       throw new FileNotFoundException("HFileOutputFormat dir " +
141           hfofDir + " not found");
142     }
143 
144     FileStatus[] familyDirStatuses = fs.listStatus(hfofDir);
145     if (familyDirStatuses == null) {
146       throw new FileNotFoundException("No families found in " + hfofDir);
147     }
148 
149     for (FileStatus stat : familyDirStatuses) {
150       if (!stat.isDir()) {
151         LOG.warn("Skipping non-directory " + stat.getPath());
152         continue;
153       }
154       Path familyDir = stat.getPath();
155       // Skip _logs, etc
156       if (familyDir.getName().startsWith("_")) continue;
157       byte[] family = familyDir.getName().getBytes();
158       Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
159       for (Path hfile : hfiles) {
160         if (hfile.getName().startsWith("_")) continue;
161         ret.add(new LoadQueueItem(family, hfile));
162       }
163     }
164   }
165 
166   /**
167    * Perform a bulk load of the given directory into the given
168    * pre-existing table.  This method is not threadsafe.
169    * 
170    * @param hfofDir the directory that was provided as the output path
171    * of a job using HFileOutputFormat
172    * @param table the table to load into
173    * @throws TableNotFoundException if table does not yet exist
174    */
175   public void doBulkLoad(Path hfofDir, final HTable table)
176     throws TableNotFoundException, IOException
177   {
178     final HConnection conn = table.getConnection();
179 
180     if (!conn.isTableAvailable(table.getTableName())) {
181       throw new TableNotFoundException("Table " +
182           Bytes.toStringBinary(table.getTableName()) +
183           "is not currently available.");
184     }
185 
186     // initialize thread pools
187     int nrThreads = cfg.getInt("hbase.loadincremental.threads.max",
188         Runtime.getRuntime().availableProcessors());
189     ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
190     builder.setNameFormat("LoadIncrementalHFiles-%1$d");
191     ExecutorService pool = new ThreadPoolExecutor(nrThreads, nrThreads,
192         60, TimeUnit.SECONDS,
193         new LinkedBlockingQueue<Runnable>(),
194         builder.build());
195     ((ThreadPoolExecutor)pool).allowCoreThreadTimeOut(true);
196 
197     // LQI queue does not need to be threadsafe -- all operations on this queue
198     // happen in this thread
199     Deque<LoadQueueItem> queue = new LinkedList<LoadQueueItem>();
200     try {
201       discoverLoadQueue(queue, hfofDir);
202       int count = 0;
203 
204       if (queue.isEmpty()) {
205         LOG.warn("Bulk load operation did not find any files to load in " +
206             "directory " + hfofDir.toUri() + ".  Does it contain files in " +
207             "subdirectories that correspond to column family names?");
208         return;
209       }
210 
211       if (queue.isEmpty()) {
212         LOG.warn("Bulk load operation did not find any files to load in " +
213         "directory " + hfofDir.toUri() + ".  Does it contain files in " +
214         "subdirectories that correspond to column family names?");
215       }
216 
217       // Assumes that region splits can happen while this occurs.
218       while (!queue.isEmpty()) {
219         // need to reload split keys each iteration.
220         final Pair<byte[][], byte[][]> startEndKeys = table.getStartEndKeys();
221         if (count != 0) {
222           LOG.info("Split occured while grouping HFiles, retry attempt " +
223               + count + " with " + queue.size() + " files remaining to group or split");
224         }
225 
226         int maxRetries = cfg.getInt("hbase.bulkload.retries.number", 0);
227         if (maxRetries != 0 && count >= maxRetries) {
228           LOG.error("Retry attempted " + count +  " times without completing, bailing out");
229           return;
230         }
231         count++;
232 
233         // Using ByteBuffer for byte[] equality semantics
234         Multimap<ByteBuffer, LoadQueueItem> regionGroups = groupOrSplitPhase(table,
235             pool, queue, startEndKeys);
236 
237         bulkLoadPhase(table, conn, pool, queue, regionGroups);
238 
239         // NOTE: The next iteration's split / group could happen in parallel to
240         // atomic bulkloads assuming that there are splits and no merges, and
241         // that we can atomically pull out the groups we want to retry.
242       }
243 
244     } finally {
245       pool.shutdown();
246       if (queue != null && !queue.isEmpty()) {
247         StringBuilder err = new StringBuilder();
248         err.append("-------------------------------------------------\n");
249         err.append("Bulk load aborted with some files not yet loaded:\n");
250         err.append("-------------------------------------------------\n");
251         for (LoadQueueItem q : queue) {
252           err.append("  ").append(q.hfilePath).append('\n');
253         }
254         LOG.error(err);
255       }
256     }
257   }
258 
259   /**
260    * This takes the LQI's grouped by likely regions and attempts to bulk load
261    * them.  Any failures are re-queued for another pass with the
262    * groupOrSplitPhase.
263    */
264   protected void bulkLoadPhase(final HTable table, final HConnection conn,
265       ExecutorService pool, Deque<LoadQueueItem> queue,
266       final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException {
267     // atomically bulk load the groups.
268     Set<Future<List<LoadQueueItem>>> loadingFutures = new HashSet<Future<List<LoadQueueItem>>>();
269     for (Entry<ByteBuffer, ? extends Collection<LoadQueueItem>> e: regionGroups.asMap().entrySet()) {
270       final byte[] first = e.getKey().array();
271       final Collection<LoadQueueItem> lqis =  e.getValue();
272 
273       final Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
274         public List<LoadQueueItem> call() throws Exception {
275           List<LoadQueueItem> toRetry = tryAtomicRegionLoad(conn, table.getTableName(), first, lqis);
276           return toRetry;
277         }
278       };
279       loadingFutures.add(pool.submit(call));
280     }
281 
282     // get all the results.
283     for (Future<List<LoadQueueItem>> future : loadingFutures) {
284       try {
285         List<LoadQueueItem> toRetry = future.get();
286 
287         // LQIs that are requeued to be regrouped.
288         queue.addAll(toRetry);
289 
290       } catch (ExecutionException e1) {
291         Throwable t = e1.getCause();
292         if (t instanceof IOException) {
293           // At this point something unrecoverable has happened.
294           // TODO Implement bulk load recovery
295           throw new IOException("BulkLoad encountered an unrecoverable problem", t);
296         }
297         LOG.error("Unexpected execution exception during bulk load", e1);
298         throw new IllegalStateException(t);
299       } catch (InterruptedException e1) {
300         LOG.error("Unexpected interrupted exception during bulk load", e1);
301         throw new IllegalStateException(e1);
302       }
303     }
304   }
305 
306   /**
307    * @return A Multimap<startkey, LoadQueueItem> that groups LQI by likely
308    * bulk load region targets.
309    */
310   private Multimap<ByteBuffer, LoadQueueItem> groupOrSplitPhase(final HTable table,
311       ExecutorService pool, Deque<LoadQueueItem> queue,
312       final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
313     // <region start key, LQI> need synchronized only within this scope of this
314     // phase because of the puts that happen in futures.
315     Multimap<ByteBuffer, LoadQueueItem> rgs = HashMultimap.create();
316     final Multimap<ByteBuffer, LoadQueueItem> regionGroups = Multimaps.synchronizedMultimap(rgs);
317 
318     // drain LQIs and figure out bulk load groups
319     Set<Future<List<LoadQueueItem>>> splittingFutures = new HashSet<Future<List<LoadQueueItem>>>();
320     while (!queue.isEmpty()) {
321       final LoadQueueItem item = queue.remove();
322       
323       final Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
324         public List<LoadQueueItem> call() throws Exception {
325           List<LoadQueueItem> splits = groupOrSplit(regionGroups, item, table, startEndKeys);
326           return splits;
327         }
328       };
329       splittingFutures.add(pool.submit(call));
330     }
331     // get all the results.  All grouping and splitting must finish before
332     // we can attempt the atomic loads.
333     for (Future<List<LoadQueueItem>> lqis : splittingFutures) {
334       try {
335         List<LoadQueueItem> splits = lqis.get();
336         if (splits != null) {
337           queue.addAll(splits);
338         }
339       } catch (ExecutionException e1) {
340         Throwable t = e1.getCause();
341         if (t instanceof IOException) {
342           LOG.error("IOException during splitting", e1);
343           throw (IOException)t; // would have been thrown if not parallelized,
344         }
345         LOG.error("Unexpected execution exception during splitting", e1);
346         throw new IllegalStateException(t);
347       } catch (InterruptedException e1) {
348         LOG.error("Unexpected interrupted exception during splitting", e1);
349         throw new IllegalStateException(e1);
350       }
351     }
352     return regionGroups;
353   }
354 
355   // unique file name for the table
356   String getUniqueName(byte[] tableName) {
357     String name = Bytes.toStringBinary(tableName) + "," + regionCount.incrementAndGet();
358     return name;
359   }
360 
361   protected List<LoadQueueItem> splitStoreFile(final LoadQueueItem item,
362       final HTable table, byte[] startKey,
363       byte[] splitKey) throws IOException {
364     final Path hfilePath = item.hfilePath;
365 
366     // We use a '_' prefix which is ignored when walking directory trees
367     // above.
368     final Path tmpDir = new Path(item.hfilePath.getParent(), "_tmp");
369 
370     LOG.info("HFile at " + hfilePath + " no longer fits inside a single " +
371     "region. Splitting...");
372 
373     String uniqueName = getUniqueName(table.getTableName());
374     HColumnDescriptor familyDesc = table.getTableDescriptor().getFamily(item.family);
375     Path botOut = new Path(tmpDir, uniqueName + ".bottom");
376     Path topOut = new Path(tmpDir, uniqueName + ".top");
377     splitStoreFile(getConf(), hfilePath, familyDesc, splitKey,
378         botOut, topOut);
379 
380     // Add these back at the *front* of the queue, so there's a lower
381     // chance that the region will just split again before we get there.
382     List<LoadQueueItem> lqis = new ArrayList<LoadQueueItem>(2);
383     lqis.add(new LoadQueueItem(item.family, botOut));
384     lqis.add(new LoadQueueItem(item.family, topOut));
385 
386     LOG.info("Successfully split into new HFiles " + botOut + " and " + topOut);
387     return lqis;
388   }
389 
390   /**
391    * Attempt to assign the given load queue item into its target region group.
392    * If the hfile boundary no longer fits into a region, physically splits
393    * the hfile such that the new bottom half will fit and returns the list of
394    * LQI's corresponding to the resultant hfiles.
395    *
396    * protected for testing
397    */
398   protected List<LoadQueueItem> groupOrSplit(Multimap<ByteBuffer, LoadQueueItem> regionGroups,
399       final LoadQueueItem item, final HTable table,
400       final Pair<byte[][], byte[][]> startEndKeys)
401       throws IOException {
402     final Path hfilePath = item.hfilePath;
403     final FileSystem fs = hfilePath.getFileSystem(getConf());
404     HFile.Reader hfr = HFile.createReader(fs, hfilePath,
405         new CacheConfig(getConf()));
406     final byte[] first, last;
407     try {
408       hfr.loadFileInfo();
409       first = hfr.getFirstRowKey();
410       last = hfr.getLastRowKey();
411     }  finally {
412       hfr.close();
413     }
414 
415     LOG.info("Trying to load hfile=" + hfilePath +
416         " first=" + Bytes.toStringBinary(first) +
417         " last="  + Bytes.toStringBinary(last));
418     if (first == null || last == null) {
419       assert first == null && last == null;
420       // TODO what if this is due to a bad HFile?
421       LOG.info("hfile " + hfilePath + " has no entries, skipping");
422       return null;
423     }
424     if (Bytes.compareTo(first, last) > 0) {
425       throw new IllegalArgumentException(
426       "Invalid range: " + Bytes.toStringBinary(first) +
427       " > " + Bytes.toStringBinary(last));
428     }
429     int idx = Arrays.binarySearch(startEndKeys.getFirst(), first,
430         Bytes.BYTES_COMPARATOR);
431     if (idx < 0) {
432       // not on boundary, returns -(insertion index).  Calculate region it
433       // would be in.
434       idx = -(idx + 1) - 1;
435     }
436     final int indexForCallable = idx;
437     boolean lastKeyInRange =
438       Bytes.compareTo(last, startEndKeys.getSecond()[idx]) < 0 ||
439       Bytes.equals(startEndKeys.getSecond()[idx], HConstants.EMPTY_BYTE_ARRAY);
440     if (!lastKeyInRange) {
441       List<LoadQueueItem> lqis = splitStoreFile(item, table,
442           startEndKeys.getFirst()[indexForCallable],
443           startEndKeys.getSecond()[indexForCallable]);
444       return lqis;
445     }
446 
447     // group regions.
448     regionGroups.put(ByteBuffer.wrap(startEndKeys.getFirst()[idx]), item);
449     return null;
450   }
451 
452   /**
453    * Attempts to do an atomic load of many hfiles into a region.  If it fails,
454    * it returns a list of hfiles that need to be retried.  If it is successful
455    * it will return an empty list.
456    * 
457    * NOTE: To maintain row atomicity guarantees, region server callable should
458    * succeed atomically and fails atomically.
459    * 
460    * Protected for testing.
461    * 
462    * @return empty list if success, list of items to retry on recoverable
463    * failure
464    */
465   protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn,
466       byte[] tableName, final byte[] first, Collection<LoadQueueItem> lqis) throws IOException {
467 
468     final List<Pair<byte[], String>> famPaths =
469       new ArrayList<Pair<byte[], String>>(lqis.size());
470     for (LoadQueueItem lqi : lqis) {
471       famPaths.add(Pair.newPair(lqi.family, lqi.hfilePath.toString()));
472     }
473 
474     final ServerCallable<Boolean> svrCallable = new ServerCallable<Boolean>(conn,
475         tableName, first) {
476       @Override
477       public Boolean call() throws Exception {
478         LOG.debug("Going to connect to server " + location + " for row "
479             + Bytes.toStringBinary(row));
480         byte[] regionName = location.getRegionInfo().getRegionName();
481         return server.bulkLoadHFiles(famPaths, regionName);
482       }
483     };
484 
485     try {
486       List<LoadQueueItem> toRetry = new ArrayList<LoadQueueItem>();
487       boolean success = conn.getRegionServerWithRetries(svrCallable);
488       if (!success) {
489         LOG.warn("Attempt to bulk load region containing "
490             + Bytes.toStringBinary(first) + " into table "
491             + Bytes.toStringBinary(tableName)  + " with files " + lqis
492             + " failed.  This is recoverable and they will be retried.");
493         toRetry.addAll(lqis); // return lqi's to retry
494       }
495       // success
496       return toRetry;
497     } catch (IOException e) {
498       LOG.error("Encountered unrecoverable error from region server", e);
499       throw e;
500     }
501   }
502 
503   /**
504    * Split a storefile into a top and bottom half, maintaining
505    * the metadata, recreating bloom filters, etc.
506    */
507   static void splitStoreFile(
508       Configuration conf, Path inFile,
509       HColumnDescriptor familyDesc, byte[] splitKey,
510       Path bottomOut, Path topOut) throws IOException
511   {
512     // Open reader with no block cache, and not in-memory
513     Reference topReference = new Reference(splitKey, Range.top);
514     Reference bottomReference = new Reference(splitKey, Range.bottom);
515 
516     copyHFileHalf(conf, inFile, topOut, topReference, familyDesc);
517     copyHFileHalf(conf, inFile, bottomOut, bottomReference, familyDesc);
518   }
519 
520   /**
521    * Copy half of an HFile into a new HFile.
522    */
523   private static void copyHFileHalf(
524       Configuration conf, Path inFile, Path outFile, Reference reference,
525       HColumnDescriptor familyDescriptor)
526   throws IOException {
527     FileSystem fs = inFile.getFileSystem(conf);
528     CacheConfig cacheConf = new CacheConfig(conf);
529     HalfStoreFileReader halfReader = null;
530     StoreFile.Writer halfWriter = null;
531     try {
532       halfReader = new HalfStoreFileReader(fs, inFile, cacheConf,
533           reference);
534       Map<byte[], byte[]> fileInfo = halfReader.loadFileInfo();
535 
536       int blocksize = familyDescriptor.getBlocksize();
537       Algorithm compression = familyDescriptor.getCompression();
538       BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
539 
540       halfWriter = new StoreFile.Writer(
541           fs, outFile, blocksize, compression, conf, cacheConf,
542           KeyValue.COMPARATOR, bloomFilterType, 0);
543       HFileScanner scanner = halfReader.getScanner(false, false);
544       scanner.seekTo();
545       do {
546         KeyValue kv = scanner.getKeyValue();
547         halfWriter.append(kv);
548       } while (scanner.next());
549 
550       for (Map.Entry<byte[],byte[]> entry : fileInfo.entrySet()) {
551         if (shouldCopyHFileMetaKey(entry.getKey())) {
552           halfWriter.appendFileInfo(entry.getKey(), entry.getValue());
553         }
554       }
555     } finally {
556       if (halfWriter != null) halfWriter.close();
557       if (halfReader != null) halfReader.close(cacheConf.shouldEvictOnClose());
558     }
559   }
560 
561   private static boolean shouldCopyHFileMetaKey(byte[] key) {
562     return !HFile.isReservedFileInfoKey(key);
563   }
564 
565   private boolean doesTableExist(String tableName) throws Exception {
566     return hbAdmin.tableExists(tableName);
567   }
568   
569   /*
570    * Infers region boundaries for a new table.
571    * Parameter:
572    *   bdryMap is a map between keys to an integer belonging to {+1, -1}
573    *     If a key is a start key of a file, then it maps to +1
574    *     If a key is an end key of a file, then it maps to -1
575    * Algo:
576    * 1) Poll on the keys in order: 
577    *    a) Keep adding the mapped values to these keys (runningSum) 
578    *    b) Each time runningSum reaches 0, add the start Key from when the runningSum had started to a boundary list.
579    * 2) Return the boundary list. 
580    */
581   public static byte[][] inferBoundaries(TreeMap<byte[], Integer> bdryMap) {
582     ArrayList<byte[]> keysArray = new ArrayList<byte[]>();
583     int runningValue = 0;
584     byte[] currStartKey = null;
585     boolean firstBoundary = true;
586     
587     for (Map.Entry<byte[], Integer> item: bdryMap.entrySet()) {
588       if (runningValue == 0) currStartKey = item.getKey();
589       runningValue += item.getValue();
590       if (runningValue == 0) {
591         if (!firstBoundary) keysArray.add(currStartKey);
592         firstBoundary = false;
593       } 
594     }
595     
596     return keysArray.toArray(new byte[0][0]);
597   }
598  
599   /*
600    * If the table is created for the first time, then "completebulkload" reads the files twice.
601    * More modifications necessary if we want to avoid doing it.
602    */
603   private void createTable(String tableName, String dirPath) throws Exception {
604     Path hfofDir = new Path(dirPath);
605     FileSystem fs = hfofDir.getFileSystem(getConf());
606 
607     if (!fs.exists(hfofDir)) {
608       throw new FileNotFoundException("HFileOutputFormat dir " +
609           hfofDir + " not found");
610     }
611 
612     FileStatus[] familyDirStatuses = fs.listStatus(hfofDir);
613     if (familyDirStatuses == null) {
614       throw new FileNotFoundException("No families found in " + hfofDir);
615     }
616 
617     HTableDescriptor htd = new HTableDescriptor(tableName);
618     HColumnDescriptor hcd = null;
619 
620     // Add column families
621     // Build a set of keys
622     byte[][] keys = null;
623     TreeMap<byte[], Integer> map = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
624     
625     for (FileStatus stat : familyDirStatuses) {
626       if (!stat.isDir()) {
627         LOG.warn("Skipping non-directory " + stat.getPath());
628         continue;
629       }
630       Path familyDir = stat.getPath();
631       // Skip _logs, etc
632       if (familyDir.getName().startsWith("_")) continue;
633       byte[] family = familyDir.getName().getBytes();
634      
635       hcd = new HColumnDescriptor(family);
636       htd.addFamily(hcd);
637       
638       Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
639       for (Path hfile : hfiles) {
640         if (hfile.getName().startsWith("_")) continue;
641         
642         HFile.Reader reader = HFile.createReader(fs, hfile,
643             new CacheConfig(getConf()));
644         final byte[] first, last;
645         try {
646           reader.loadFileInfo();
647           first = reader.getFirstRowKey();
648           last =  reader.getLastRowKey();
649 
650           LOG.info("Trying to figure out region boundaries hfile=" + hfile +
651             " first=" + Bytes.toStringBinary(first) +
652             " last="  + Bytes.toStringBinary(last));
653           
654           // To eventually infer start key-end key boundaries
655           Integer value = map.containsKey(first)?(Integer)map.get(first):0;
656           map.put(first, value+1);
657 
658           value = map.containsKey(last)?(Integer)map.get(last):0;
659           map.put(last, value-1);
660         }  finally {
661           reader.close();
662         }
663       }
664     }
665     
666     keys = LoadIncrementalHFiles.inferBoundaries(map);
667     try {    
668       this.hbAdmin.createTableAsync(htd, keys);
669     } catch (java.net.SocketTimeoutException e) {
670       System.err.println("Caught Socket timeout.. Mostly caused by a slow region assignment by master!");
671     }
672 
673     HTable table = new HTable(this.cfg, tableName);
674 
675     HConnection conn = table.getConnection();
676     int ctr = 0;
677     while (!conn.isTableAvailable(table.getTableName()) && (ctr<TABLE_CREATE_MAX_RETRIES)) {
678       LOG.info("Table " + tableName + "not yet available... Sleeping for 60 more seconds...");
679       /* Every TABLE_CREATE_SLEEP milliseconds, wakes up and checks if the table is available*/
680       Thread.sleep(TABLE_CREATE_SLEEP);
681       ctr++;
682     }
683     LOG.info("Table "+ tableName +" is finally available!!");
684   }
685 
686   @Override
687   public int run(String[] args) throws Exception {
688     if (args.length != 2) {
689       usage();
690       return -1;
691     }
692 
693     String dirPath   = args[0];
694     String tableName = args[1];
695 
696     boolean tableExists   = this.doesTableExist(tableName);
697     if (!tableExists) this.createTable(tableName,dirPath);
698 
699     Path hfofDir = new Path(dirPath);
700     HTable table = new HTable(this.cfg, tableName);
701 
702     doBulkLoad(hfofDir, table);
703     return 0;
704   }
705 
706   public static void main(String[] args) throws Exception {
707     int ret = ToolRunner.run(new LoadIncrementalHFiles(HBaseConfiguration.create()), args);
708     System.exit(ret);
709   }
710 
711 }