View Javadoc

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