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