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