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