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