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