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 java.io.FileNotFoundException;
24  import java.io.IOException;
25  import java.io.InterruptedIOException;
26  import java.nio.ByteBuffer;
27  import java.util.ArrayList;
28  import java.util.Arrays;
29  import java.util.Collection;
30  import java.util.Deque;
31  import java.util.HashMap;
32  import java.util.HashSet;
33  import java.util.Iterator;
34  import java.util.LinkedList;
35  import java.util.List;
36  import java.util.Map;
37  import java.util.Map.Entry;
38  import java.util.Set;
39  import java.util.TreeMap;
40  import java.util.UUID;
41  import java.util.concurrent.Callable;
42  import java.util.concurrent.ExecutionException;
43  import java.util.concurrent.ExecutorService;
44  import java.util.concurrent.Future;
45  import java.util.concurrent.LinkedBlockingQueue;
46  import java.util.concurrent.ThreadPoolExecutor;
47  import java.util.concurrent.TimeUnit;
48
49  import org.apache.commons.lang.mutable.MutableInt;
50  import org.apache.commons.logging.Log;
51  import org.apache.commons.logging.LogFactory;
52  import org.apache.hadoop.conf.Configuration;
53  import org.apache.hadoop.conf.Configured;
54  import org.apache.hadoop.fs.FileStatus;
55  import org.apache.hadoop.fs.FileSystem;
56  import org.apache.hadoop.fs.Path;
57  import org.apache.hadoop.fs.permission.FsPermission;
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.TableName;
63  import org.apache.hadoop.hbase.TableNotFoundException;
64  import org.apache.hadoop.hbase.classification.InterfaceAudience;
65  import org.apache.hadoop.hbase.classification.InterfaceStability;
66  import org.apache.hadoop.hbase.client.Admin;
67  import org.apache.hadoop.hbase.client.Connection;
68  import org.apache.hadoop.hbase.client.ConnectionFactory;
69  import org.apache.hadoop.hbase.client.RegionLocator;
70  import org.apache.hadoop.hbase.client.RegionServerCallable;
71  import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
72  import org.apache.hadoop.hbase.client.SecureBulkLoadClient;
73  import org.apache.hadoop.hbase.client.Table;
74  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
75  import org.apache.hadoop.hbase.io.HFileLink;
76  import org.apache.hadoop.hbase.io.HalfStoreFileReader;
77  import org.apache.hadoop.hbase.io.Reference;
78  import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
79  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
80  import org.apache.hadoop.hbase.io.hfile.HFile;
81  import org.apache.hadoop.hbase.io.hfile.HFileContext;
82  import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
83  import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
84  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
85  import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
86  import org.apache.hadoop.hbase.regionserver.BloomType;
87  import org.apache.hadoop.hbase.regionserver.HStore;
88  import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
89  import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
90  import org.apache.hadoop.hbase.security.UserProvider;
91  import org.apache.hadoop.hbase.security.token.FsDelegationToken;
92  import org.apache.hadoop.hbase.util.Bytes;
93  import org.apache.hadoop.hbase.util.FSHDFSUtils;
94  import org.apache.hadoop.hbase.util.Pair;
95  import org.apache.hadoop.util.Tool;
96  import org.apache.hadoop.util.ToolRunner;
97
98  import com.google.common.collect.HashMultimap;
99  import com.google.common.collect.Multimap;
100 import com.google.common.collect.Multimaps;
101 import com.google.common.util.concurrent.ThreadFactoryBuilder;
102
103 /**
104  * Tool to load the output of HFileOutputFormat into an existing table.
105  */
106 @InterfaceAudience.Public
107 @InterfaceStability.Stable
108 public class LoadIncrementalHFiles extends Configured implements Tool {
109   private static final Log LOG = LogFactory.getLog(LoadIncrementalHFiles.class);
110   private boolean initalized = false;
111
112   public static final String NAME = "completebulkload";
113   public static final String MAX_FILES_PER_REGION_PER_FAMILY
114     = "hbase.mapreduce.bulkload.max.hfiles.perRegion.perFamily";
115   private static final String ASSIGN_SEQ_IDS = "hbase.mapreduce.bulkload.assign.sequenceNumbers";
116   public final static String CREATE_TABLE_CONF_KEY = "create.table";
117   public final static String SILENCE_CONF_KEY = "ignore.unmatched.families";
118   public final static String ALWAYS_COPY_FILES = "always.copy.files";
119
120   // We use a '.' prefix which is ignored when walking directory trees
121   // above. It is invalid family name.
122   final static String TMP_DIR = ".tmp";
123
124   private int maxFilesPerRegionPerFamily;
125   private boolean assignSeqIds;
126   private Set<String> unmatchedFamilies = new HashSet<String>();
127
128   // Source filesystem
129   private FileSystem fs;
130   // Source delegation token
131   private FsDelegationToken fsDelegationToken;
132   private String bulkToken;
133   private UserProvider userProvider;
134   private int nrThreads;
135   private RpcControllerFactory rpcControllerFactory;
136
137   public LoadIncrementalHFiles(Configuration conf) throws Exception {
138     super(conf);
139     this.rpcControllerFactory = new RpcControllerFactory(conf);
140     initialize();
141   }
142
143   private void initialize() throws Exception {
144     if (initalized) {
145       return;
146     }
147     // make a copy, just to be sure we're not overriding someone else's config
148     setConf(HBaseConfiguration.create(getConf()));
149     Configuration conf = getConf();
150     // disable blockcache for tool invocation, see HBASE-10500
151     conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0);
152     this.userProvider = UserProvider.instantiate(conf);
153     this.fsDelegationToken = new FsDelegationToken(userProvider, "renewer");
154     assignSeqIds = conf.getBoolean(ASSIGN_SEQ_IDS, true);
155     maxFilesPerRegionPerFamily = conf.getInt(MAX_FILES_PER_REGION_PER_FAMILY, 32);
156     nrThreads = conf.getInt("hbase.loadincremental.threads.max",
157       Runtime.getRuntime().availableProcessors());
158     initalized = true;
159   }
160
161   private void usage() {
162     System.err.println("usage: " + NAME + " /path/to/hfileoutputformat-output tablename" + "\n -D"
163         + CREATE_TABLE_CONF_KEY + "=no - can be used to avoid creation of table by this tool\n"
164         + "  Note: if you set this to 'no', then the target table must already exist in HBase\n -D"
165         + SILENCE_CONF_KEY + "=yes - can be used to ignore unmatched column families\n"
166         + "\n");
167   }
168
169   private interface BulkHFileVisitor<TFamily> {
170     TFamily bulkFamily(final byte[] familyName)
171       throws IOException;
172     void bulkHFile(final TFamily family, final FileStatus hfileStatus)
173       throws IOException;
174   }
175
176   /**
177    * Iterate over the bulkDir hfiles.
178    * Skip reference, HFileLink, files starting with "_" and non-valid hfiles.
179    */
180   private static <TFamily> void visitBulkHFiles(final FileSystem fs, final Path bulkDir,
181     final BulkHFileVisitor<TFamily> visitor) throws IOException {
182     visitBulkHFiles(fs, bulkDir, visitor, true);
183   }
184
185   /**
186    * Iterate over the bulkDir hfiles.
187    * Skip reference, HFileLink, files starting with "_".
188    * Check and skip non-valid hfiles by default, or skip this validation by setting
189    * 'hbase.loadincremental.validate.hfile' to false.
190    */
191   private static <TFamily> void visitBulkHFiles(final FileSystem fs, final Path bulkDir,
192     final BulkHFileVisitor<TFamily> visitor, final boolean validateHFile) throws IOException {
193     if (!fs.exists(bulkDir)) {
194       throw new FileNotFoundException("Bulkload dir " + bulkDir + " not found");
195     }
196
197     FileStatus[] familyDirStatuses = fs.listStatus(bulkDir);
198     if (familyDirStatuses == null) {
199       throw new FileNotFoundException("No families found in " + bulkDir);
200     }
201
202     for (FileStatus familyStat : familyDirStatuses) {
203       if (!familyStat.isDirectory()) {
204         LOG.warn("Skipping non-directory " + familyStat.getPath());
205         continue;
206       }
207       Path familyDir = familyStat.getPath();
208       byte[] familyName = familyDir.getName().getBytes();
209       // Skip invalid family
210       try {
211         HColumnDescriptor.isLegalFamilyName(familyName);
212       }
213       catch (IllegalArgumentException e) {
214         LOG.warn("Skipping invalid " + familyStat.getPath());
215         continue;
216       }
217       TFamily family = visitor.bulkFamily(familyName);
218
219       FileStatus[] hfileStatuses = fs.listStatus(familyDir);
220       for (FileStatus hfileStatus : hfileStatuses) {
221         if (!fs.isFile(hfileStatus.getPath())) {
222           LOG.warn("Skipping non-file " + hfileStatus);
223           continue;
224         }
225
226         Path hfile = hfileStatus.getPath();
227         // Skip "_", reference, HFileLink
228         String fileName = hfile.getName();
229         if (fileName.startsWith("_")) {
230           continue;
231         }
232         if (StoreFileInfo.isReference(fileName)) {
233           LOG.warn("Skipping reference " + fileName);
234           continue;
235         }
236         if (HFileLink.isHFileLink(fileName)) {
237           LOG.warn("Skipping HFileLink " + fileName);
238           continue;
239         }
240
241         // Validate HFile Format if needed
242         if (validateHFile) {
243           try {
244             if (!HFile.isHFileFormat(fs, hfile)) {
245               LOG.warn("the file " + hfile + " doesn't seems to be an hfile. skipping");
246               continue;
247             }
248           } catch (FileNotFoundException e) {
249             LOG.warn("the file " + hfile + " was removed");
250             continue;
251           }
252 	}
253
254         visitor.bulkHFile(family, hfileStatus);
255       }
256     }
257   }
258
259   /**
260    * Represents an HFile waiting to be loaded. An queue is used
261    * in this class in order to support the case where a region has
262    * split during the process of the load. When this happens,
263    * the HFile is split into two physical parts across the new
264    * region boundary, and each part is added back into the queue.
265    * The import process finishes when the queue is empty.
266    */
267   public static class LoadQueueItem {
268     final byte[] family;
269     final Path hfilePath;
270
271     public LoadQueueItem(byte[] family, Path hfilePath) {
272       this.family = family;
273       this.hfilePath = hfilePath;
274     }
275
276     @Override
277     public String toString() {
278       return "family:"+ Bytes.toString(family) + " path:" + hfilePath.toString();
279     }
280   }
281
282   /*
283    * Populate the Queue with given HFiles
284    */
285   private void populateLoadQueue(final Deque<LoadQueueItem> ret,
286       Map<byte[], List<Path>> map) throws IOException {
287     for (Map.Entry<byte[], List<Path>> entry : map.entrySet()) {
288       for (Path p : entry.getValue()) {
289         ret.add(new LoadQueueItem(entry.getKey(), p));
290       }
291     }
292   }
293
294   /**
295    * Walk the given directory for all HFiles, and return a Queue
296    * containing all such files.
297    */
298   private void discoverLoadQueue(final Deque<LoadQueueItem> ret, final Path hfofDir,
299       final boolean validateHFile) throws IOException {
300     fs = hfofDir.getFileSystem(getConf());
301     visitBulkHFiles(fs, hfofDir, new BulkHFileVisitor<byte[]>() {
302       @Override
303       public byte[] bulkFamily(final byte[] familyName) {
304         return familyName;
305       }
306       @Override
307       public void bulkHFile(final byte[] family, final FileStatus hfile) throws IOException {
308         long length = hfile.getLen();
309         if (length > getConf().getLong(HConstants.HREGION_MAX_FILESIZE,
310             HConstants.DEFAULT_MAX_FILE_SIZE)) {
311           LOG.warn("Trying to bulk load hfile " + hfile.getPath() + " with size: " +
312               length + " bytes can be problematic as it may lead to oversplitting.");
313         }
314         ret.add(new LoadQueueItem(family, hfile.getPath()));
315       }
316     }, validateHFile);
317   }
318
319   /**
320    * Perform a bulk load of the given directory into the given
321    * pre-existing table.  This method is not threadsafe.
322    *
323    * @param hfofDir the directory that was provided as the output path
324    *   of a job using HFileOutputFormat
325    * @param admin the Admin
326    * @param table the table to load into
327    * @param regionLocator region locator
328    * @throws TableNotFoundException if table does not yet exist
329    */
330   public void doBulkLoad(Path hfofDir, final Admin admin, Table table,
331       RegionLocator regionLocator) throws TableNotFoundException, IOException {
332     doBulkLoad(hfofDir, admin, table, regionLocator, false, false);
333   }
334
335   void cleanup(Admin admin, Deque<LoadQueueItem> queue, ExecutorService pool,
336       SecureBulkLoadClient secureClient) throws IOException {
337     fsDelegationToken.releaseDelegationToken();
338     if (bulkToken != null && secureClient != null) {
339       secureClient.cleanupBulkLoad(admin.getConnection(), bulkToken);
340     }
341     if (pool != null) {
342       pool.shutdown();
343     }
344     if (!queue.isEmpty()) {
345       StringBuilder err = new StringBuilder();
346       err.append("-------------------------------------------------\n");
347       err.append("Bulk load aborted with some files not yet loaded:\n");
348       err.append("-------------------------------------------------\n");
349       for (LoadQueueItem q : queue) {
350         err.append("  ").append(q.hfilePath).append('\n');
351       }
352       LOG.error(err);
353     }
354   }
355   /**
356    * Perform a bulk load of the given directory into the given
357    * pre-existing table.  This method is not threadsafe.
358    *
359    * @param map map of family to List of hfiles
360    * @param admin the Admin
361    * @param table the table to load into
362    * @param regionLocator region locator
363    * @param silence true to ignore unmatched column families
364    * @param copyFile always copy hfiles if true
365    * @throws TableNotFoundException if table does not yet exist
366    */
367   public void doBulkLoad(Map<byte[], List<Path>> map, final Admin admin, Table table,
368           RegionLocator regionLocator, boolean silence, boolean copyFile)
369               throws TableNotFoundException, IOException {
370     if (!admin.isTableAvailable(regionLocator.getName())) {
371       throw new TableNotFoundException("Table " + table.getName() + " is not currently available.");
372     }
373     // LQI queue does not need to be threadsafe -- all operations on this queue
374     // happen in this thread
375     Deque<LoadQueueItem> queue = new LinkedList<>();
376     ExecutorService pool = null;
377     SecureBulkLoadClient secureClient = null;
378     try {
379       prepareHFileQueue(map, table, queue, silence);
380       if (queue.isEmpty()) {
381         LOG.warn("Bulk load operation did not get any files to load");
382         return;
383       }
384       pool = createExecutorService();
385       secureClient = new SecureBulkLoadClient(table.getConfiguration(), table);
386       for (Map.Entry<byte[], List<Path>> entry : map.entrySet()) {
387         for (Path p : entry.getValue()) {
388           fs = p.getFileSystem(table.getConfiguration());
389           break;
390         }
391       }
392       performBulkLoad(admin, table, regionLocator, queue, pool, secureClient, copyFile);
393     } finally {
394       cleanup(admin, queue, pool, secureClient);
395     }
396   }
397
398   /**
399    * Perform a bulk load of the given directory into the given
400    * pre-existing table.  This method is not threadsafe.
401    *
402    * @param hfofDir the directory that was provided as the output path
403    *   of a job using HFileOutputFormat
404    * @param admin the Admin
405    * @param table the table to load into
406    * @param regionLocator region locator
407    * @param silence true to ignore unmatched column families
408    * @param copyFile always copy hfiles if true
409    * @throws TableNotFoundException if table does not yet exist
410    */
411   public void doBulkLoad(Path hfofDir, final Admin admin, Table table,
412       RegionLocator regionLocator, boolean silence, boolean copyFile)
413           throws TableNotFoundException, IOException {
414     if (!admin.isTableAvailable(regionLocator.getName())) {
415       throw new TableNotFoundException("Table " + table.getName() + " is not currently available.");
416     }
417
418     /*
419      * Checking hfile format is a time-consuming operation, we should have an option to skip
420      * this step when bulkloading millions of HFiles. See HBASE-13985.
421      */
422     boolean validateHFile = getConf().getBoolean("hbase.loadincremental.validate.hfile", true);
423     if (!validateHFile) {
424       LOG.warn("You are skipping HFiles validation, it might cause some data loss if files " +
425           "are not correct. If you fail to read data from your table after using this " +
426           "option, consider removing the files and bulkload again without this option. " +
427           "See HBASE-13985");
428     }
429     // LQI queue does not need to be threadsafe -- all operations on this queue
430     // happen in this thread
431     Deque<LoadQueueItem> queue = new LinkedList<>();
432     ExecutorService pool = null;
433     SecureBulkLoadClient secureClient = null;
434     try {
435       prepareHFileQueue(hfofDir, table, queue, validateHFile, silence);
436
437       if (queue.isEmpty()) {
438         LOG.warn("Bulk load operation did not find any files to load in " +
439             "directory " + hfofDir != null ? hfofDir.toUri() : "" + ".  Does it contain files in " +
440             "subdirectories that correspond to column family names?");
441         return;
442       }
443       pool = createExecutorService();
444       secureClient = new SecureBulkLoadClient(table.getConfiguration(), table);
445       performBulkLoad(admin, table, regionLocator, queue, pool, secureClient, copyFile);
446     } finally {
447       cleanup(admin, queue, pool, secureClient);
448     }
449   }
450
451   void performBulkLoad(final Admin admin, Table table, RegionLocator regionLocator,
452       Deque<LoadQueueItem> queue, ExecutorService pool,
453       SecureBulkLoadClient secureClient, boolean copyFile) throws IOException {
454     int count = 0;
455
456     if(isSecureBulkLoadEndpointAvailable()) {
457       LOG.warn("SecureBulkLoadEndpoint is deprecated. It will be removed in future releases.");
458       LOG.warn("Secure bulk load has been integrated into HBase core.");
459     }
460
461     //If using secure bulk load, get source delegation token, and
462     //prepare staging directory and token
463     // fs is the source filesystem
464     fsDelegationToken.acquireDelegationToken(fs);
465     bulkToken = secureClient.prepareBulkLoad(admin.getConnection());
466
467     // Assumes that region splits can happen while this occurs.
468     while (!queue.isEmpty()) {
469       // need to reload split keys each iteration.
470       final Pair<byte[][], byte[][]> startEndKeys = regionLocator.getStartEndKeys();
471       if (count != 0) {
472         LOG.info("Split occured while grouping HFiles, retry attempt " +
473             + count + " with " + queue.size() + " files remaining to group or split");
474       }
475
476       int maxRetries = getConf().getInt(HConstants.BULKLOAD_MAX_RETRIES_NUMBER, 10);
477       maxRetries = Math.max(maxRetries, startEndKeys.getFirst().length + 1);
478       if (maxRetries != 0 && count >= maxRetries) {
479         throw new IOException("Retry attempted " + count +
480             " times without completing, bailing out");
481       }
482       count++;
483
484       // Using ByteBuffer for byte[] equality semantics
485       Multimap<ByteBuffer, LoadQueueItem> regionGroups = groupOrSplitPhase(table,
486           pool, queue, startEndKeys);
487
488       if (!checkHFilesCountPerRegionPerFamily(regionGroups)) {
489         // Error is logged inside checkHFilesCountPerRegionPerFamily.
490         throw new IOException("Trying to load more than " + maxFilesPerRegionPerFamily
491             + " hfiles to one family of one region");
492       }
493
494       bulkLoadPhase(table, admin.getConnection(), pool, queue, regionGroups, copyFile);
495
496       // NOTE: The next iteration's split / group could happen in parallel to
497       // atomic bulkloads assuming that there are splits and no merges, and
498       // that we can atomically pull out the groups we want to retry.
499     }
500
501     if (!queue.isEmpty()) {
502       throw new RuntimeException("Bulk load aborted with some files not yet loaded."
503         + "Please check log for more details.");
504     }
505   }
506
507   /**
508    * Prepare a collection of {@link LoadQueueItem} from list of source hfiles contained in the
509    * passed directory and validates whether the prepared queue has all the valid table column
510    * families in it.
511    * @param hfilesDir directory containing list of hfiles to be loaded into the table
512    * @param table table to which hfiles should be loaded
513    * @param queue queue which needs to be loaded into the table
514    * @param validateHFile if true hfiles will be validated for its format
515    * @throws IOException If any I/O or network error occurred
516    */
517   public void prepareHFileQueue(Path hfilesDir, Table table, Deque<LoadQueueItem> queue,
518       boolean validateHFile) throws IOException {
519     prepareHFileQueue(hfilesDir, table, queue, validateHFile, false);
520   }
521
522   /**
523    * Prepare a collection of {@link LoadQueueItem} from list of source hfiles contained in the
524    * passed directory and validates whether the prepared queue has all the valid table column
525    * families in it.
526    * @param hfilesDir directory containing list of hfiles to be loaded into the table
527    * @param table table to which hfiles should be loaded
528    * @param queue queue which needs to be loaded into the table
529    * @param validateHFile if true hfiles will be validated for its format
530    * @param silence  true to ignore unmatched column families
531    * @throws IOException If any I/O or network error occurred
532    */
533   public void prepareHFileQueue(Path hfilesDir, Table table,
534       Deque<LoadQueueItem> queue, boolean validateHFile, boolean silence) throws IOException {
535     discoverLoadQueue(queue, hfilesDir, validateHFile);
536     validateFamiliesInHFiles(table, queue, silence);
537   }
538
539   /**
540    * Prepare a collection of {@link LoadQueueItem} from list of source hfiles contained in the
541    * passed directory and validates whether the prepared queue has all the valid table column
542    * families in it.
543    * @param map map of family to List of hfiles
544    * @param table table to which hfiles should be loaded
545    * @param queue queue which needs to be loaded into the table
546    * @param silence  true to ignore unmatched column families
547    * @throws IOException If any I/O or network error occurred
548    */
549   public void prepareHFileQueue(Map<byte[], List<Path>> map, Table table,
550       Deque<LoadQueueItem> queue, boolean silence) throws IOException {
551     populateLoadQueue(queue, map);
552     validateFamiliesInHFiles(table, queue, silence);
553   }
554
555   // Initialize a thread pool
556   private ExecutorService createExecutorService() {
557     ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
558     builder.setNameFormat("LoadIncrementalHFiles-%1$d");
559     ExecutorService pool = new ThreadPoolExecutor(nrThreads, nrThreads, 60, TimeUnit.SECONDS,
560         new LinkedBlockingQueue<Runnable>(), builder.build());
561     ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
562     return pool;
563   }
564
565   /**
566    * Checks whether there is any invalid family name in HFiles to be bulk loaded.
567    */
568   private void validateFamiliesInHFiles(Table table, Deque<LoadQueueItem> queue, boolean silence)
569       throws IOException {
570     Collection<HColumnDescriptor> families = table.getTableDescriptor().getFamilies();
571     List<String> familyNames = new ArrayList<>(families.size());
572     for (HColumnDescriptor family : families) {
573       familyNames.add(family.getNameAsString());
574     }
575     Iterator<LoadQueueItem> queueIter = queue.iterator();
576     while (queueIter.hasNext()) {
577       LoadQueueItem lqi = queueIter.next();
578       String familyNameInHFile = Bytes.toString(lqi.family);
579       if (!familyNames.contains(familyNameInHFile)) {
580         unmatchedFamilies.add(familyNameInHFile);
581       }
582     }
583     if (unmatchedFamilies.size() > 0) {
584       String msg =
585           "Unmatched family names found: unmatched family names in HFiles to be bulkloaded: "
586               + unmatchedFamilies + "; valid family names of table " + table.getName() + " are: "
587               + familyNames;
588       LOG.error(msg);
589       if (!silence) throw new IOException(msg);
590     }
591   }
592
593   /**
594    * Used by the replication sink to load the hfiles from the source cluster. It does the following,
595    * <ol>
596    * <li>LoadIncrementalHFiles#groupOrSplitPhase(Table, ExecutorService, Deque, Pair)}</li>
597    * <li>LoadIncrementalHFiles#bulkLoadPhase(Table, Connection, ExecutorService, Deque, Multimap)
598    * </li>
599    * </ol>
600    * @param table Table to which these hfiles should be loaded to
601    * @param conn Connection to use
602    * @param queue {@link LoadQueueItem} has hfiles yet to be loaded
603    * @param startEndKeys starting and ending row keys of the region
604    */
605   public void loadHFileQueue(final Table table, final Connection conn, Deque<LoadQueueItem> queue,
606       Pair<byte[][], byte[][]> startEndKeys) throws IOException {
607     loadHFileQueue(table, conn, queue, startEndKeys, false);
608   }
609
610   /**
611    * Used by the replication sink to load the hfiles from the source cluster. It does the following,
612    * <ol>
613    * <li>LoadIncrementalHFiles#groupOrSplitPhase(Table, ExecutorService, Deque, Pair)}</li>
614    * <li>LoadIncrementalHFiles#bulkLoadPhase(Table, Connection, ExecutorService, Deque, Multimap)
615    * </li>
616    * </ol>
617    * @param table Table to which these hfiles should be loaded to
618    * @param conn Connection to use
619    * @param queue {@link LoadQueueItem} has hfiles yet to be loaded
620    * @param startEndKeys starting and ending row keys of the region
621    */
622   public void loadHFileQueue(final Table table, final Connection conn, Deque<LoadQueueItem> queue,
623       Pair<byte[][], byte[][]> startEndKeys, boolean copyFile) throws IOException {
624     ExecutorService pool = null;
625     try {
626       pool = createExecutorService();
627       Multimap<ByteBuffer, LoadQueueItem> regionGroups =
628           groupOrSplitPhase(table, pool, queue, startEndKeys);
629       bulkLoadPhase(table, conn, pool, queue, regionGroups, copyFile);
630     } finally {
631       if (pool != null) {
632         pool.shutdown();
633       }
634     }
635   }
636
637   /**
638    * This takes the LQI's grouped by likely regions and attempts to bulk load
639    * them.  Any failures are re-queued for another pass with the
640    * groupOrSplitPhase.
641    */
642   protected void bulkLoadPhase(final Table table, final Connection conn,
643       ExecutorService pool, Deque<LoadQueueItem> queue,
644       final Multimap<ByteBuffer, LoadQueueItem> regionGroups, boolean copyFile) throws IOException {
645     // atomically bulk load the groups.
646     Set<Future<List<LoadQueueItem>>> loadingFutures = new HashSet<>();
647     for (Entry<ByteBuffer, ? extends Collection<LoadQueueItem>> e: regionGroups.asMap().entrySet()){
648       final byte[] first = e.getKey().array();
649       final Collection<LoadQueueItem> lqis =  e.getValue();
650
651       final Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
652         @Override
653         public List<LoadQueueItem> call() throws Exception {
654           List<LoadQueueItem> toRetry =
655               tryAtomicRegionLoad(conn, table.getName(), first, lqis, copyFile);
656           return toRetry;
657         }
658       };
659       loadingFutures.add(pool.submit(call));
660     }
661
662     // get all the results.
663     for (Future<List<LoadQueueItem>> future : loadingFutures) {
664       try {
665         List<LoadQueueItem> toRetry = future.get();
666
667         // LQIs that are requeued to be regrouped.
668         queue.addAll(toRetry);
669
670       } catch (ExecutionException e1) {
671         Throwable t = e1.getCause();
672         if (t instanceof IOException) {
673           // At this point something unrecoverable has happened.
674           // TODO Implement bulk load recovery
675           throw new IOException("BulkLoad encountered an unrecoverable problem", t);
676         }
677         LOG.error("Unexpected execution exception during bulk load", e1);
678         throw new IllegalStateException(t);
679       } catch (InterruptedException e1) {
680         LOG.error("Unexpected interrupted exception during bulk load", e1);
681         throw (InterruptedIOException)new InterruptedIOException().initCause(e1);
682       }
683     }
684   }
685
686   private boolean checkHFilesCountPerRegionPerFamily(
687       final Multimap<ByteBuffer, LoadQueueItem> regionGroups) {
688     for (Entry<ByteBuffer,
689       ? extends Collection<LoadQueueItem>> e: regionGroups.asMap().entrySet()) {
690       final Collection<LoadQueueItem> lqis =  e.getValue();
691       HashMap<byte[], MutableInt> filesMap = new HashMap<>();
692       for (LoadQueueItem lqi: lqis) {
693         MutableInt count = filesMap.get(lqi.family);
694         if (count == null) {
695           count = new MutableInt();
696           filesMap.put(lqi.family, count);
697         }
698         count.increment();
699         if (count.intValue() > maxFilesPerRegionPerFamily) {
700           LOG.error("Trying to load more than " + maxFilesPerRegionPerFamily
701             + " hfiles to family " + Bytes.toStringBinary(lqi.family)
702             + " of region with start key "
703             + Bytes.toStringBinary(e.getKey()));
704           return false;
705         }
706       }
707     }
708     return true;
709   }
710
711   /**
712    * @return A map that groups LQI by likely bulk load region targets.
713    */
714   private Multimap<ByteBuffer, LoadQueueItem> groupOrSplitPhase(final Table table,
715       ExecutorService pool, Deque<LoadQueueItem> queue,
716       final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
717     // <region start key, LQI> need synchronized only within this scope of this
718     // phase because of the puts that happen in futures.
719     Multimap<ByteBuffer, LoadQueueItem> rgs = HashMultimap.create();
720     final Multimap<ByteBuffer, LoadQueueItem> regionGroups = Multimaps.synchronizedMultimap(rgs);
721
722     // drain LQIs and figure out bulk load groups
723     Set<Future<List<LoadQueueItem>>> splittingFutures = new HashSet<>();
724     while (!queue.isEmpty()) {
725       final LoadQueueItem item = queue.remove();
726
727       final Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
728         @Override
729         public List<LoadQueueItem> call() throws Exception {
730           List<LoadQueueItem> splits = groupOrSplit(regionGroups, item, table, startEndKeys);
731           return splits;
732         }
733       };
734       splittingFutures.add(pool.submit(call));
735     }
736     // get all the results.  All grouping and splitting must finish before
737     // we can attempt the atomic loads.
738     for (Future<List<LoadQueueItem>> lqis : splittingFutures) {
739       try {
740         List<LoadQueueItem> splits = lqis.get();
741         if (splits != null) {
742           queue.addAll(splits);
743         }
744       } catch (ExecutionException e1) {
745         Throwable t = e1.getCause();
746         if (t instanceof IOException) {
747           LOG.error("IOException during splitting", e1);
748           throw (IOException)t; // would have been thrown if not parallelized,
749         }
750         LOG.error("Unexpected execution exception during splitting", e1);
751         throw new IllegalStateException(t);
752       } catch (InterruptedException e1) {
753         LOG.error("Unexpected interrupted exception during splitting", e1);
754         throw (InterruptedIOException)new InterruptedIOException().initCause(e1);
755       }
756     }
757     return regionGroups;
758   }
759
760   // unique file name for the table
761   private String getUniqueName() {
762     return UUID.randomUUID().toString().replaceAll("-", "");
763   }
764
765   protected List<LoadQueueItem> splitStoreFile(final LoadQueueItem item,
766       final Table table, byte[] startKey,
767       byte[] splitKey) throws IOException {
768     final Path hfilePath = item.hfilePath;
769
770     Path tmpDir = item.hfilePath.getParent();
771     if (!tmpDir.getName().equals(TMP_DIR)) {
772       tmpDir = new Path(tmpDir, TMP_DIR);
773     }
774
775     LOG.info("HFile at " + hfilePath + " no longer fits inside a single " +
776       "region. Splitting...");
777
778     String uniqueName = getUniqueName();
779     HColumnDescriptor familyDesc = table.getTableDescriptor().getFamily(item.family);
780
781     Path botOut = new Path(tmpDir, uniqueName + ".bottom");
782     Path topOut = new Path(tmpDir, uniqueName + ".top");
783     splitStoreFile(getConf(), hfilePath, familyDesc, splitKey, botOut, topOut);
784
785     FileSystem fs = tmpDir.getFileSystem(getConf());
786     fs.setPermission(tmpDir, FsPermission.valueOf("-rwxrwxrwx"));
787     fs.setPermission(botOut, FsPermission.valueOf("-rwxrwxrwx"));
788     fs.setPermission(topOut, FsPermission.valueOf("-rwxrwxrwx"));
789
790     // Add these back at the *front* of the queue, so there's a lower
791     // chance that the region will just split again before we get there.
792     List<LoadQueueItem> lqis = new ArrayList<LoadQueueItem>(2);
793     lqis.add(new LoadQueueItem(item.family, botOut));
794     lqis.add(new LoadQueueItem(item.family, topOut));
795
796     // If the current item is already the result of previous splits,
797     // we don't need it anymore. Clean up to save space.
798     // It is not part of the original input files.
799     try {
800       tmpDir = item.hfilePath.getParent();
801       if (tmpDir.getName().equals(TMP_DIR)) {
802         fs.delete(item.hfilePath, false);
803       }
804     } catch (IOException e) {
805       LOG.warn("Unable to delete temporary split file " + item.hfilePath);
806     }
807     LOG.info("Successfully split into new HFiles " + botOut + " and " + topOut);
808     return lqis;
809   }
810
811   /**
812    * Attempt to assign the given load queue item into its target region group.
813    * If the hfile boundary no longer fits into a region, physically splits
814    * the hfile such that the new bottom half will fit and returns the list of
815    * LQI's corresponding to the resultant hfiles.
816    *
817    * protected for testing
818    * @throws IOException if an IO failure is encountered
819    */
820   protected List<LoadQueueItem> groupOrSplit(Multimap<ByteBuffer, LoadQueueItem> regionGroups,
821       final LoadQueueItem item, final Table table,
822       final Pair<byte[][], byte[][]> startEndKeys)
823       throws IOException {
824     final Path hfilePath = item.hfilePath;
825     // fs is the source filesystem
826     if (fs == null) {
827       fs = hfilePath.getFileSystem(getConf());
828     }
829     HFile.Reader hfr = HFile.createReader(fs, hfilePath,
830         new CacheConfig(getConf()), getConf());
831     final byte[] first, last;
832     try {
833       hfr.loadFileInfo();
834       first = hfr.getFirstRowKey();
835       last = hfr.getLastRowKey();
836     }  finally {
837       hfr.close();
838     }
839
840     LOG.info("Trying to load hfile=" + hfilePath +
841         " first=" + Bytes.toStringBinary(first) +
842         " last="  + Bytes.toStringBinary(last));
843     if (first == null || last == null) {
844       assert first == null && last == null;
845       // TODO what if this is due to a bad HFile?
846       LOG.info("hfile " + hfilePath + " has no entries, skipping");
847       return null;
848     }
849     if (Bytes.compareTo(first, last) > 0) {
850       throw new IllegalArgumentException(
851       "Invalid range: " + Bytes.toStringBinary(first) +
852       " > " + Bytes.toStringBinary(last));
853     }
854     int idx = Arrays.binarySearch(startEndKeys.getFirst(), first,
855         Bytes.BYTES_COMPARATOR);
856     if (idx < 0) {
857       // not on boundary, returns -(insertion index).  Calculate region it
858       // would be in.
859       idx = -(idx + 1) - 1;
860     }
861     final int indexForCallable = idx;
862
863     /**
864      * we can consider there is a region hole in following conditions. 1) if idx < 0,then first
865      * region info is lost. 2) if the endkey of a region is not equal to the startkey of the next
866      * region. 3) if the endkey of the last region is not empty.
867      */
868     if (indexForCallable < 0) {
869       throw new IOException("The first region info for table "
870           + table.getName()
871           + " cann't be found in hbase:meta.Please use hbck tool to fix it first.");
872     } else if ((indexForCallable == startEndKeys.getFirst().length - 1)
873         && !Bytes.equals(startEndKeys.getSecond()[indexForCallable], HConstants.EMPTY_BYTE_ARRAY)) {
874       throw new IOException("The last region info for table "
875           + table.getName()
876           + " cann't be found in hbase:meta.Please use hbck tool to fix it first.");
877     } else if (indexForCallable + 1 < startEndKeys.getFirst().length
878         && !(Bytes.compareTo(startEndKeys.getSecond()[indexForCallable],
879           startEndKeys.getFirst()[indexForCallable + 1]) == 0)) {
880       throw new IOException("The endkey of one region for table "
881           + table.getName()
882           + " is not equal to the startkey of the next region in hbase:meta."
883           + "Please use hbck tool to fix it first.");
884     }
885
886     boolean lastKeyInRange =
887       Bytes.compareTo(last, startEndKeys.getSecond()[idx]) < 0 ||
888       Bytes.equals(startEndKeys.getSecond()[idx], HConstants.EMPTY_BYTE_ARRAY);
889     if (!lastKeyInRange) {
890       List<LoadQueueItem> lqis = splitStoreFile(item, table,
891           startEndKeys.getFirst()[indexForCallable],
892           startEndKeys.getSecond()[indexForCallable]);
893       return lqis;
894     }
895
896     // group regions.
897     regionGroups.put(ByteBuffer.wrap(startEndKeys.getFirst()[idx]), item);
898     return null;
899   }
900
901   /**
902    * Attempts to do an atomic load of many hfiles into a region.  If it fails,
903    * it returns a list of hfiles that need to be retried.  If it is successful
904    * it will return an empty list.
905    *
906    * NOTE: To maintain row atomicity guarantees, region server callable should
907    * succeed atomically and fails atomically.
908    *
909    * Protected for testing.
910    *
911    * @return empty list if success, list of items to retry on recoverable
912    *   failure
913    */
914   protected List<LoadQueueItem> tryAtomicRegionLoad(final Connection conn,
915       final TableName tableName, final byte[] first, final Collection<LoadQueueItem> lqis,
916       boolean copyFile) throws IOException {
917     final List<Pair<byte[], String>> famPaths = new ArrayList<>(lqis.size());
918     for (LoadQueueItem lqi : lqis) {
919       if (!unmatchedFamilies.contains(Bytes.toString(lqi.family))) {
920         famPaths.add(Pair.newPair(lqi.family, lqi.hfilePath.toString()));
921       }
922     }
923     final RegionServerCallable<Boolean> svrCallable = new RegionServerCallable<Boolean>(conn,
924         rpcControllerFactory, tableName, first) {
925       @Override
926       protected Boolean rpcCall() throws Exception {
927         SecureBulkLoadClient secureClient = null;
928         boolean success = false;
929         try {
930           LOG.debug("Going to connect to server " + getLocation() + " for row "
931               + Bytes.toStringBinary(getRow()) + " with hfile group " + famPaths);
932           byte[] regionName = getLocation().getRegionInfo().getRegionName();
933           try (Table table = conn.getTable(getTableName())) {
934             secureClient = new SecureBulkLoadClient(getConf(), table);
935             success = secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
936                   assignSeqIds, fsDelegationToken.getUserToken(), bulkToken, copyFile);
937           }
938           return success;
939         } finally {
940           //Best effort copying of files that might not have been imported
941           //from the staging directory back to original location
942           //in user directory
943           if (secureClient != null && !success) {
944             FileSystem targetFs = FileSystem.get(getConf());
945          // fs is the source filesystem
946             if(fs == null) {
947               fs = lqis.iterator().next().hfilePath.getFileSystem(getConf());
948             }
949             // Check to see if the source and target filesystems are the same
950             // If they are the same filesystem, we will try move the files back
951             // because previously we moved them to the staging directory.
952             if (FSHDFSUtils.isSameHdfs(getConf(), fs, targetFs)) {
953               for(Pair<byte[], String> el : famPaths) {
954                 Path hfileStagingPath = null;
955                 Path hfileOrigPath = new Path(el.getSecond());
956                 try {
957                   hfileStagingPath= new Path(new Path(bulkToken, Bytes.toString(el.getFirst())),
958                     hfileOrigPath.getName());
959                   if(targetFs.rename(hfileStagingPath, hfileOrigPath)) {
960                     LOG.debug("Moved back file " + hfileOrigPath + " from " +
961                         hfileStagingPath);
962                   } else if(targetFs.exists(hfileStagingPath)){
963                     LOG.debug("Unable to move back file " + hfileOrigPath + " from " +
964                         hfileStagingPath);
965                   }
966                 } catch(Exception ex) {
967                   LOG.debug("Unable to move back file " + hfileOrigPath + " from " +
968                       hfileStagingPath, ex);
969                 }
970               }
971             }
972           }
973         }
974       }
975     };
976
977     try {
978       List<LoadQueueItem> toRetry = new ArrayList<>();
979       Configuration conf = getConf();
980       boolean success = RpcRetryingCallerFactory.instantiate(conf,
981           null).<Boolean> newCaller()
982           .callWithRetries(svrCallable, Integer.MAX_VALUE);
983       if (!success) {
984         LOG.warn("Attempt to bulk load region containing "
985             + Bytes.toStringBinary(first) + " into table "
986             + tableName  + " with files " + lqis
987             + " failed.  This is recoverable and they will be retried.");
988         toRetry.addAll(lqis); // return lqi's to retry
989       }
990       // success
991       return toRetry;
992     } catch (IOException e) {
993       LOG.error("Encountered unrecoverable error from region server, additional details: "
994           + svrCallable.getExceptionMessageAdditionalDetail(), e);
995       throw e;
996     }
997   }
998
999   private boolean isSecureBulkLoadEndpointAvailable() {
1000     String classes = getConf().get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
1001     return classes.contains("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
1002   }
1003
1004   /**
1005    * Split a storefile into a top and bottom half, maintaining
1006    * the metadata, recreating bloom filters, etc.
1007    */
1008   static void splitStoreFile(
1009       Configuration conf, Path inFile,
1010       HColumnDescriptor familyDesc, byte[] splitKey,
1011       Path bottomOut, Path topOut) throws IOException {
1012     // Open reader with no block cache, and not in-memory
1013     Reference topReference = Reference.createTopReference(splitKey);
1014     Reference bottomReference = Reference.createBottomReference(splitKey);
1015
1016     copyHFileHalf(conf, inFile, topOut, topReference, familyDesc);
1017     copyHFileHalf(conf, inFile, bottomOut, bottomReference, familyDesc);
1018   }
1019
1020   /**
1021    * Copy half of an HFile into a new HFile.
1022    */
1023   private static void copyHFileHalf(
1024       Configuration conf, Path inFile, Path outFile, Reference reference,
1025       HColumnDescriptor familyDescriptor)
1026   throws IOException {
1027     FileSystem fs = inFile.getFileSystem(conf);
1028     CacheConfig cacheConf = new CacheConfig(conf);
1029     HalfStoreFileReader halfReader = null;
1030     StoreFileWriter halfWriter = null;
1031     try {
1032       halfReader = new HalfStoreFileReader(fs, inFile, cacheConf, reference, conf);
1033       Map<byte[], byte[]> fileInfo = halfReader.loadFileInfo();
1034
1035       int blocksize = familyDescriptor.getBlocksize();
1036       Algorithm compression = familyDescriptor.getCompressionType();
1037       BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
1038       HFileContext hFileContext = new HFileContextBuilder()
1039                                   .withCompression(compression)
1040                                   .withChecksumType(HStore.getChecksumType(conf))
1041                                   .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf))
1042                                   .withBlockSize(blocksize)
1043                                   .withDataBlockEncoding(familyDescriptor.getDataBlockEncoding())
1044                                   .withIncludesTags(true)
1045                                   .build();
1046       halfWriter = new StoreFileWriter.Builder(conf, cacheConf,
1047           fs)
1048               .withFilePath(outFile)
1049               .withBloomType(bloomFilterType)
1050               .withFileContext(hFileContext)
1051               .build();
1052       HFileScanner scanner = halfReader.getScanner(false, false, false);
1053       scanner.seekTo();
1054       do {
1055         halfWriter.append(scanner.getCell());
1056       } while (scanner.next());
1057
1058       for (Map.Entry<byte[],byte[]> entry : fileInfo.entrySet()) {
1059         if (shouldCopyHFileMetaKey(entry.getKey())) {
1060           halfWriter.appendFileInfo(entry.getKey(), entry.getValue());
1061         }
1062       }
1063     } finally {
1064       if (halfWriter != null) {
1065         halfWriter.close();
1066       }
1067       if (halfReader != null) {
1068         halfReader.close(cacheConf.shouldEvictOnClose());
1069       }
1070     }
1071   }
1072
1073   private static boolean shouldCopyHFileMetaKey(byte[] key) {
1074     // skip encoding to keep hfile meta consistent with data block info, see HBASE-15085
1075     if (Bytes.equals(key, HFileDataBlockEncoder.DATA_BLOCK_ENCODING)) {
1076       return false;
1077     }
1078
1079     return !HFile.isReservedFileInfoKey(key);
1080   }
1081
1082   /*
1083    * Infers region boundaries for a new table.
1084    * Parameter:
1085    *   bdryMap is a map between keys to an integer belonging to {+1, -1}
1086    *     If a key is a start key of a file, then it maps to +1
1087    *     If a key is an end key of a file, then it maps to -1
1088    * Algo:
1089    * 1) Poll on the keys in order:
1090    *    a) Keep adding the mapped values to these keys (runningSum)
1091    *    b) Each time runningSum reaches 0, add the start Key from when the runningSum had started to
1092    *       a boundary list.
1093    * 2) Return the boundary list.
1094    */
1095   public static byte[][] inferBoundaries(TreeMap<byte[], Integer> bdryMap) {
1096     ArrayList<byte[]> keysArray = new ArrayList<>();
1097     int runningValue = 0;
1098     byte[] currStartKey = null;
1099     boolean firstBoundary = true;
1100
1101     for (Map.Entry<byte[], Integer> item: bdryMap.entrySet()) {
1102       if (runningValue == 0) {
1103         currStartKey = item.getKey();
1104       }
1105       runningValue += item.getValue();
1106       if (runningValue == 0) {
1107         if (!firstBoundary) {
1108           keysArray.add(currStartKey);
1109         }
1110         firstBoundary = false;
1111       }
1112     }
1113
1114     return keysArray.toArray(new byte[0][0]);
1115   }
1116
1117   /*
1118    * If the table is created for the first time, then "completebulkload" reads the files twice.
1119    * More modifications necessary if we want to avoid doing it.
1120    */
1121   private void createTable(TableName tableName, String dirPath, Admin admin) throws Exception {
1122     final Path hfofDir = new Path(dirPath);
1123     final FileSystem fs = hfofDir.getFileSystem(getConf());
1124
1125     // Add column families
1126     // Build a set of keys
1127     final HTableDescriptor htd = new HTableDescriptor(tableName);
1128     final TreeMap<byte[], Integer> map = new TreeMap<>(Bytes.BYTES_COMPARATOR);
1129     visitBulkHFiles(fs, hfofDir, new BulkHFileVisitor<HColumnDescriptor>() {
1130       @Override
1131       public HColumnDescriptor bulkFamily(final byte[] familyName) {
1132         HColumnDescriptor hcd = new HColumnDescriptor(familyName);
1133         htd.addFamily(hcd);
1134         return hcd;
1135       }
1136       @Override
1137       public void bulkHFile(final HColumnDescriptor hcd, final FileStatus hfileStatus)
1138           throws IOException {
1139         Path hfile = hfileStatus.getPath();
1140         HFile.Reader reader = HFile.createReader(fs, hfile,
1141             new CacheConfig(getConf()), getConf());
1142         try {
1143           if (hcd.getCompressionType() != reader.getFileContext().getCompression()) {
1144             hcd.setCompressionType(reader.getFileContext().getCompression());
1145             LOG.info("Setting compression " + hcd.getCompressionType().name() +
1146                      " for family " + hcd.toString());
1147           }
1148           reader.loadFileInfo();
1149           byte[] first = reader.getFirstRowKey();
1150           byte[] last  = reader.getLastRowKey();
1151
1152           LOG.info("Trying to figure out region boundaries hfile=" + hfile +
1153             " first=" + Bytes.toStringBinary(first) +
1154             " last="  + Bytes.toStringBinary(last));
1155
1156           // To eventually infer start key-end key boundaries
1157           Integer value = map.containsKey(first)? map.get(first):0;
1158           map.put(first, value+1);
1159
1160           value = map.containsKey(last)? map.get(last):0;
1161           map.put(last, value-1);
1162         } finally {
1163           reader.close();
1164         }
1165       }
1166     });
1167
1168     byte[][] keys = LoadIncrementalHFiles.inferBoundaries(map);
1169     admin.createTable(htd, keys);
1170
1171     LOG.info("Table "+ tableName +" is available!!");
1172   }
1173
1174   public int run(String dirPath, Map<byte[], List<Path>> map, TableName tableName) throws Exception{
1175     initialize();
1176     try (Connection connection = ConnectionFactory.createConnection(getConf());
1177         Admin admin = connection.getAdmin()) {
1178
1179       boolean tableExists = admin.tableExists(tableName);
1180       if (!tableExists) {
1181         if (dirPath != null && "yes".equalsIgnoreCase(getConf().get(CREATE_TABLE_CONF_KEY, "yes"))) {
1182           this.createTable(tableName, dirPath, admin);
1183         } else {
1184           String errorMsg = format("Table '%s' does not exist.", tableName);
1185           LOG.error(errorMsg);
1186           throw new TableNotFoundException(errorMsg);
1187         }
1188       }
1189       Path hfofDir = null;
1190       if (dirPath != null) {
1191         hfofDir = new Path(dirPath);
1192       }
1193
1194       try (Table table = connection.getTable(tableName);
1195         RegionLocator locator = connection.getRegionLocator(tableName)) {
1196         boolean silence = "yes".equalsIgnoreCase(getConf().get(SILENCE_CONF_KEY, ""));
1197         boolean copyFiles = "yes".equalsIgnoreCase(getConf().get(ALWAYS_COPY_FILES, ""));
1198         if (dirPath != null) {
1199           doBulkLoad(hfofDir, admin, table, locator, silence, copyFiles);
1200         } else {
1201           doBulkLoad(map, admin, table, locator, silence, copyFiles);
1202         }
1203       }
1204     }
1205
1206     return 0;
1207   }
1208
1209   @Override
1210   public int run(String[] args) throws Exception {
1211     if (args.length < 2) {
1212       usage();
1213       return -1;
1214     }
1215
1216     String dirPath = args[0];
1217     TableName tableName = TableName.valueOf(args[1]);
1218     return run(dirPath, null, tableName);
1219   }
1220
1221   public static void main(String[] args) throws Exception {
1222     Configuration conf = HBaseConfiguration.create();
1223     int ret = ToolRunner.run(conf, new LoadIncrementalHFiles(conf), args);
1224     System.exit(ret);
1225   }
1226
1227   /**
1228    * Called from replication sink, where it manages bulkToken(staging directory) by itself. This is
1229    * used only when SecureBulkLoadEndpoint is configured in hbase.coprocessor.region.classes
1230    * property. This directory is used as a temporary directory where all files are initially
1231    * copied/moved from user given directory, set all the required file permissions and then from
1232    * their it is finally loaded into a table. This should be set only when, one would like to manage
1233    * the staging directory by itself. Otherwise this tool will handle this by itself.
1234    * @param stagingDir staging directory path
1235    */
1236   public void setBulkToken(String stagingDir) {
1237     this.bulkToken = stagingDir;
1238   }
1239 }