View Javadoc

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