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