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   static class LoadQueueItem {
268     final byte[] family;
269     final Path hfilePath;
270 
271     public LoadQueueItem(byte[] family, Path hfilePath) {
272       this.family = family;
273       this.hfilePath = hfilePath;
274     }
275 
276     @Override
277     public String toString() {
278       return "family:"+ Bytes.toString(family) + " path:" + hfilePath.toString();
279     }
280   }
281 
282   /**
283    * Walk the given directory for all HFiles, and return a Queue
284    * containing all such files.
285    */
286   private void discoverLoadQueue(final Deque<LoadQueueItem> ret, final Path hfofDir,
287     final boolean validateHFile) throws IOException {
288     fs = hfofDir.getFileSystem(getConf());
289     visitBulkHFiles(fs, hfofDir, new BulkHFileVisitor<byte[]>() {
290       @Override
291       public byte[] bulkFamily(final byte[] familyName) {
292         return familyName;
293       }
294       @Override
295       public void bulkHFile(final byte[] family, final FileStatus hfile) throws IOException {
296         long length = hfile.getLen();
297         if (length > getConf().getLong(HConstants.HREGION_MAX_FILESIZE,
298             HConstants.DEFAULT_MAX_FILE_SIZE)) {
299           LOG.warn("Trying to bulk load hfile " + hfile.getPath() + " with size: " +
300               length + " bytes can be problematic as it may lead to oversplitting.");
301         }
302         ret.add(new LoadQueueItem(family, hfile.getPath()));
303       }
304     }, validateHFile);
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    * @deprecated since hbase-1.3.0, hbase-1.2.2, and hbase-1.1.6; replaced
316    * by {@link #doBulkLoad(Path, Admin, Table, RegionLocator)}.
317    * @see HBASE-15876
318    */
319   @SuppressWarnings("deprecation")
320   @Deprecated
321   public void doBulkLoad(Path hfofDir, final HTable table)
322     throws TableNotFoundException, IOException
323   {
324     Admin admin = null;
325     Table t = table;
326     Connection conn = table.getConnection();
327     boolean closeConnWhenFinished = false;
328     try {
329       if (conn instanceof ClusterConnection && ((ClusterConnection) conn).isManaged()) {
330         LOG.warn("managed connection cannot be used for bulkload. Creating unmanaged connection.");
331         // can only use unmanaged connections from here on out.
332         conn = ConnectionFactory.createConnection(table.getConfiguration());
333         t = conn.getTable(table.getName());
334         closeConnWhenFinished = true;
335         if (conn instanceof ClusterConnection && ((ClusterConnection) conn).isManaged()) {
336           throw new RuntimeException("Failed to create unmanaged connection.");
337         }
338         admin = conn.getAdmin();
339       } else {
340         admin = conn.getAdmin();
341       }
342       try (RegionLocator rl = conn.getRegionLocator(t.getName())) {
343         doBulkLoad(hfofDir, admin, t, rl);
344       }
345     } finally {
346       if (admin != null) admin.close();
347       if (closeConnWhenFinished) {
348         t.close();
349         conn.close();
350       }
351     }
352   }
353 
354   /**
355    * Perform a bulk load of the given directory into the given
356    * pre-existing table.  This method is not threadsafe.
357    *
358    * @param hfofDir the directory that was provided as the output path
359    * of a job using HFileOutputFormat
360    * @param table the table to load into
361    * @throws TableNotFoundException if table does not yet exist
362    */
363   @SuppressWarnings("deprecation")
364   public void doBulkLoad(Path hfofDir, final Admin admin, Table table,
365       RegionLocator regionLocator) throws TableNotFoundException, IOException  {
366 
367     if (!admin.isTableAvailable(regionLocator.getName())) {
368       throw new TableNotFoundException("Table " + table.getName() + " is not currently available.");
369     }
370 
371     // initialize thread pools
372     int nrThreads = getConf().getInt("hbase.loadincremental.threads.max",
373       Runtime.getRuntime().availableProcessors());
374     ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
375     builder.setNameFormat("LoadIncrementalHFiles-%1$d");
376     ExecutorService pool = new ThreadPoolExecutor(nrThreads, nrThreads,
377         60, TimeUnit.SECONDS,
378         new LinkedBlockingQueue<Runnable>(),
379         builder.build());
380     ((ThreadPoolExecutor)pool).allowCoreThreadTimeOut(true);
381 
382     // LQI queue does not need to be threadsafe -- all operations on this queue
383     // happen in this thread
384     Deque<LoadQueueItem> queue = new LinkedList<LoadQueueItem>();
385     try {
386       /*
387        * Checking hfile format is a time-consuming operation, we should have an option to skip
388        * this step when bulkloading millions of HFiles. See HBASE-13985.
389        */
390       boolean validateHFile = getConf().getBoolean("hbase.loadincremental.validate.hfile", true);
391       if(!validateHFile) {
392         LOG.warn("You are skipping HFiles validation, it might cause some data loss if files " +
393           "are not correct. If you fail to read data from your table after using this " +
394           "option, consider removing the files and bulkload again without this option. " +
395           "See HBASE-13985");
396       }
397       discoverLoadQueue(queue, hfofDir, validateHFile);
398       // check whether there is invalid family name in HFiles to be bulkloaded
399       Collection<HColumnDescriptor> families = table.getTableDescriptor().getFamilies();
400       ArrayList<String> familyNames = new ArrayList<String>(families.size());
401       for (HColumnDescriptor family : families) {
402         familyNames.add(family.getNameAsString());
403       }
404       ArrayList<String> unmatchedFamilies = new ArrayList<String>();
405       Iterator<LoadQueueItem> queueIter = queue.iterator();
406       while (queueIter.hasNext()) {
407         LoadQueueItem lqi = queueIter.next();
408         String familyNameInHFile = Bytes.toString(lqi.family);
409         if (!familyNames.contains(familyNameInHFile)) {
410           unmatchedFamilies.add(familyNameInHFile);
411         }
412       }
413       if (unmatchedFamilies.size() > 0) {
414         String msg =
415             "Unmatched family names found: unmatched family names in HFiles to be bulkloaded: "
416                 + unmatchedFamilies + "; valid family names of table "
417                 + table.getName() + " are: " + familyNames;
418         LOG.error(msg);
419         throw new IOException(msg);
420       }
421       int count = 0;
422 
423       if (queue.isEmpty()) {
424         LOG.warn("Bulk load operation did not find any files to load in " +
425             "directory " + hfofDir.toUri() + ".  Does it contain files in " +
426             "subdirectories that correspond to column family names?");
427         return;
428       }
429 
430       //If using secure bulk load, get source delegation token, and
431       //prepare staging directory and token
432       // fs is the source filesystem
433       fsDelegationToken.acquireDelegationToken(fs);
434       if(isSecureBulkLoadEndpointAvailable()) {
435         bulkToken = new SecureBulkLoadClient(table).prepareBulkLoad(table.getName());
436       }
437 
438       // Assumes that region splits can happen while this occurs.
439       while (!queue.isEmpty()) {
440         // need to reload split keys each iteration.
441         final Pair<byte[][], byte[][]> startEndKeys = regionLocator.getStartEndKeys();
442         if (count != 0) {
443           LOG.info("Split occured while grouping HFiles, retry attempt " +
444               + count + " with " + queue.size() + " files remaining to group or split");
445         }
446 
447         int maxRetries = getConf().getInt("hbase.bulkload.retries.number", 10);
448         maxRetries = Math.max(maxRetries, startEndKeys.getFirst().length + 1);
449         if (maxRetries != 0 && count >= maxRetries) {
450           throw new IOException("Retry attempted " + count +
451             " times without completing, bailing out");
452         }
453         count++;
454 
455         // Using ByteBuffer for byte[] equality semantics
456         Multimap<ByteBuffer, LoadQueueItem> regionGroups = groupOrSplitPhase(table,
457             pool, queue, startEndKeys);
458 
459         if (!checkHFilesCountPerRegionPerFamily(regionGroups)) {
460           // Error is logged inside checkHFilesCountPerRegionPerFamily.
461           throw new IOException("Trying to load more than " + maxFilesPerRegionPerFamily
462             + " hfiles to one family of one region");
463         }
464 
465         bulkLoadPhase(table, admin.getConnection(), pool, queue, regionGroups);
466 
467         // NOTE: The next iteration's split / group could happen in parallel to
468         // atomic bulkloads assuming that there are splits and no merges, and
469         // that we can atomically pull out the groups we want to retry.
470       }
471 
472     } finally {
473       fsDelegationToken.releaseDelegationToken();
474       if(bulkToken != null) {
475         new SecureBulkLoadClient(table).cleanupBulkLoad(bulkToken);
476       }
477       pool.shutdown();
478       if (queue != null && !queue.isEmpty()) {
479         StringBuilder err = new StringBuilder();
480         err.append("-------------------------------------------------\n");
481         err.append("Bulk load aborted with some files not yet loaded:\n");
482         err.append("-------------------------------------------------\n");
483         for (LoadQueueItem q : queue) {
484           err.append("  ").append(q.hfilePath).append('\n');
485         }
486         LOG.error(err);
487       }
488     }
489 
490     if (queue != null && !queue.isEmpty()) {
491         throw new RuntimeException("Bulk load aborted with some files not yet loaded."
492           + "Please check log for more details.");
493     }
494   }
495 
496   /**
497    * This takes the LQI's grouped by likely regions and attempts to bulk load
498    * them.  Any failures are re-queued for another pass with the
499    * groupOrSplitPhase.
500    */
501   protected void bulkLoadPhase(final Table table, final Connection conn,
502       ExecutorService pool, Deque<LoadQueueItem> queue,
503       final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException {
504     // atomically bulk load the groups.
505     Set<Future<List<LoadQueueItem>>> loadingFutures = new HashSet<Future<List<LoadQueueItem>>>();
506     for (Entry<ByteBuffer, ? extends Collection<LoadQueueItem>> e: regionGroups.asMap().entrySet()) {
507       final byte[] first = e.getKey().array();
508       final Collection<LoadQueueItem> lqis =  e.getValue();
509 
510       final Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
511         @Override
512         public List<LoadQueueItem> call() throws Exception {
513           List<LoadQueueItem> toRetry =
514               tryAtomicRegionLoad(conn, table.getName(), first, lqis);
515           return toRetry;
516         }
517       };
518       loadingFutures.add(pool.submit(call));
519     }
520 
521     // get all the results.
522     for (Future<List<LoadQueueItem>> future : loadingFutures) {
523       try {
524         List<LoadQueueItem> toRetry = future.get();
525 
526         // LQIs that are requeued to be regrouped.
527         queue.addAll(toRetry);
528 
529       } catch (ExecutionException e1) {
530         Throwable t = e1.getCause();
531         if (t instanceof IOException) {
532           // At this point something unrecoverable has happened.
533           // TODO Implement bulk load recovery
534           throw new IOException("BulkLoad encountered an unrecoverable problem", t);
535         }
536         LOG.error("Unexpected execution exception during bulk load", e1);
537         throw new IllegalStateException(t);
538       } catch (InterruptedException e1) {
539         LOG.error("Unexpected interrupted exception during bulk load", e1);
540         throw (InterruptedIOException)new InterruptedIOException().initCause(e1);
541       }
542     }
543   }
544 
545   private boolean checkHFilesCountPerRegionPerFamily(
546       final Multimap<ByteBuffer, LoadQueueItem> regionGroups) {
547     for (Entry<ByteBuffer,
548         ? extends Collection<LoadQueueItem>> e: regionGroups.asMap().entrySet()) {
549       final Collection<LoadQueueItem> lqis =  e.getValue();
550       HashMap<byte[], MutableInt> filesMap = new HashMap<byte[], MutableInt>();
551       for (LoadQueueItem lqi: lqis) {
552         MutableInt count = filesMap.get(lqi.family);
553         if (count == null) {
554           count = new MutableInt();
555           filesMap.put(lqi.family, count);
556         }
557         count.increment();
558         if (count.intValue() > maxFilesPerRegionPerFamily) {
559           LOG.error("Trying to load more than " + maxFilesPerRegionPerFamily
560             + " hfiles to family " + Bytes.toStringBinary(lqi.family)
561             + " of region with start key "
562             + Bytes.toStringBinary(e.getKey()));
563           return false;
564         }
565       }
566     }
567     return true;
568   }
569 
570   /**
571    * @return A Multimap<startkey, LoadQueueItem> that groups LQI by likely
572    * bulk load region targets.
573    */
574   private Multimap<ByteBuffer, LoadQueueItem> groupOrSplitPhase(final Table table,
575       ExecutorService pool, Deque<LoadQueueItem> queue,
576       final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
577     // <region start key, LQI> need synchronized only within this scope of this
578     // phase because of the puts that happen in futures.
579     Multimap<ByteBuffer, LoadQueueItem> rgs = HashMultimap.create();
580     final Multimap<ByteBuffer, LoadQueueItem> regionGroups = Multimaps.synchronizedMultimap(rgs);
581 
582     // drain LQIs and figure out bulk load groups
583     Set<Future<List<LoadQueueItem>>> splittingFutures = new HashSet<Future<List<LoadQueueItem>>>();
584     while (!queue.isEmpty()) {
585       final LoadQueueItem item = queue.remove();
586 
587       final Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
588         @Override
589         public List<LoadQueueItem> call() throws Exception {
590           List<LoadQueueItem> splits = groupOrSplit(regionGroups, item, table, startEndKeys);
591           return splits;
592         }
593       };
594       splittingFutures.add(pool.submit(call));
595     }
596     // get all the results.  All grouping and splitting must finish before
597     // we can attempt the atomic loads.
598     for (Future<List<LoadQueueItem>> lqis : splittingFutures) {
599       try {
600         List<LoadQueueItem> splits = lqis.get();
601         if (splits != null) {
602           queue.addAll(splits);
603         }
604       } catch (ExecutionException e1) {
605         Throwable t = e1.getCause();
606         if (t instanceof IOException) {
607           LOG.error("IOException during splitting", e1);
608           throw (IOException)t; // would have been thrown if not parallelized,
609         }
610         LOG.error("Unexpected execution exception during splitting", e1);
611         throw new IllegalStateException(t);
612       } catch (InterruptedException e1) {
613         LOG.error("Unexpected interrupted exception during splitting", e1);
614         throw (InterruptedIOException)new InterruptedIOException().initCause(e1);
615       }
616     }
617     return regionGroups;
618   }
619 
620   // unique file name for the table
621   private String getUniqueName() {
622     return UUID.randomUUID().toString().replaceAll("-", "");
623   }
624 
625   protected List<LoadQueueItem> splitStoreFile(final LoadQueueItem item,
626       final Table table, byte[] startKey,
627       byte[] splitKey) throws IOException {
628     final Path hfilePath = item.hfilePath;
629 
630     Path tmpDir = item.hfilePath.getParent();
631     if (!tmpDir.getName().equals(TMP_DIR)) {
632       tmpDir = new Path(tmpDir, TMP_DIR);
633     }
634 
635     LOG.info("HFile at " + hfilePath + " no longer fits inside a single " +
636     "region. Splitting...");
637 
638     String uniqueName = getUniqueName();
639     HColumnDescriptor familyDesc = table.getTableDescriptor().getFamily(item.family);
640     Path botOut = new Path(tmpDir, uniqueName + ".bottom");
641     Path topOut = new Path(tmpDir, uniqueName + ".top");
642     splitStoreFile(getConf(), hfilePath, familyDesc, splitKey,
643         botOut, topOut);
644 
645     FileSystem fs = tmpDir.getFileSystem(getConf());
646     fs.setPermission(tmpDir, FsPermission.valueOf("-rwxrwxrwx"));
647     fs.setPermission(botOut, FsPermission.valueOf("-rwxrwxrwx"));
648     fs.setPermission(topOut, FsPermission.valueOf("-rwxrwxrwx"));
649 
650     // Add these back at the *front* of the queue, so there's a lower
651     // chance that the region will just split again before we get there.
652     List<LoadQueueItem> lqis = new ArrayList<LoadQueueItem>(2);
653     lqis.add(new LoadQueueItem(item.family, botOut));
654     lqis.add(new LoadQueueItem(item.family, topOut));
655 
656     // If the current item is already the result of previous splits,
657     // we don't need it anymore. Clean up to save space.
658     // It is not part of the original input files.
659     try {
660       tmpDir = item.hfilePath.getParent();
661       if (tmpDir.getName().equals(TMP_DIR)) {
662         fs.delete(item.hfilePath, false);
663       }
664     } catch (IOException e) {
665       LOG.warn("Unable to delete temporary split file " + item.hfilePath);
666     }
667     LOG.info("Successfully split into new HFiles " + botOut + " and " + topOut);
668     return lqis;
669   }
670 
671   /**
672    * Attempt to assign the given load queue item into its target region group.
673    * If the hfile boundary no longer fits into a region, physically splits
674    * the hfile such that the new bottom half will fit and returns the list of
675    * LQI's corresponding to the resultant hfiles.
676    *
677    * protected for testing
678    * @throws IOException
679    */
680   protected List<LoadQueueItem> groupOrSplit(Multimap<ByteBuffer, LoadQueueItem> regionGroups,
681       final LoadQueueItem item, final Table table,
682       final Pair<byte[][], byte[][]> startEndKeys)
683       throws IOException {
684     final Path hfilePath = item.hfilePath;
685     HFile.Reader hfr = HFile.createReader(fs, hfilePath,
686         new CacheConfig(getConf()), getConf());
687     final byte[] first, last;
688     try {
689       hfr.loadFileInfo();
690       first = hfr.getFirstRowKey();
691       last = hfr.getLastRowKey();
692     }  finally {
693       hfr.close();
694     }
695 
696     LOG.info("Trying to load hfile=" + hfilePath +
697         " first=" + Bytes.toStringBinary(first) +
698         " last="  + Bytes.toStringBinary(last));
699     if (first == null || last == null) {
700       assert first == null && last == null;
701       // TODO what if this is due to a bad HFile?
702       LOG.info("hfile " + hfilePath + " has no entries, skipping");
703       return null;
704     }
705     if (Bytes.compareTo(first, last) > 0) {
706       throw new IllegalArgumentException(
707       "Invalid range: " + Bytes.toStringBinary(first) +
708       " > " + Bytes.toStringBinary(last));
709     }
710     int idx = Arrays.binarySearch(startEndKeys.getFirst(), first,
711         Bytes.BYTES_COMPARATOR);
712     if (idx < 0) {
713       // not on boundary, returns -(insertion index).  Calculate region it
714       // would be in.
715       idx = -(idx + 1) - 1;
716     }
717     final int indexForCallable = idx;
718 
719     /**
720      * we can consider there is a region hole in following conditions. 1) if idx < 0,then first
721      * region info is lost. 2) if the endkey of a region is not equal to the startkey of the next
722      * region. 3) if the endkey of the last region is not empty.
723      */
724     if (indexForCallable < 0) {
725       throw new IOException("The first region info for table "
726           + table.getName()
727           + " cann't be found in hbase:meta.Please use hbck tool to fix it first.");
728     } else if ((indexForCallable == startEndKeys.getFirst().length - 1)
729         && !Bytes.equals(startEndKeys.getSecond()[indexForCallable], HConstants.EMPTY_BYTE_ARRAY)) {
730       throw new IOException("The last region info for table "
731           + table.getName()
732           + " cann't be found in hbase:meta.Please use hbck tool to fix it first.");
733     } else if (indexForCallable + 1 < startEndKeys.getFirst().length
734         && !(Bytes.compareTo(startEndKeys.getSecond()[indexForCallable],
735           startEndKeys.getFirst()[indexForCallable + 1]) == 0)) {
736       throw new IOException("The endkey of one region for table "
737           + table.getName()
738           + " is not equal to the startkey of the next region in hbase:meta."
739           + "Please use hbck tool to fix it first.");
740     }
741 
742     boolean lastKeyInRange =
743       Bytes.compareTo(last, startEndKeys.getSecond()[idx]) < 0 ||
744       Bytes.equals(startEndKeys.getSecond()[idx], HConstants.EMPTY_BYTE_ARRAY);
745     if (!lastKeyInRange) {
746       List<LoadQueueItem> lqis = splitStoreFile(item, table,
747           startEndKeys.getFirst()[indexForCallable],
748           startEndKeys.getSecond()[indexForCallable]);
749       return lqis;
750     }
751 
752     // group regions.
753     regionGroups.put(ByteBuffer.wrap(startEndKeys.getFirst()[idx]), item);
754     return null;
755   }
756 
757   /**
758    * @deprecated As of release 0.96
759    *             (<a href="https://issues.apache.org/jira/browse/HBASE-9508">HBASE-9508</a>).
760    *             This will be removed in HBase 2.0.0.
761    *             Use {@link #tryAtomicRegionLoad(Connection, TableName, byte[], Collection)}.
762    */
763   @Deprecated
764   protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn,
765       final byte [] tableName, final byte[] first, Collection<LoadQueueItem> lqis)
766   throws IOException {
767     return tryAtomicRegionLoad(conn, TableName.valueOf(tableName), first, lqis);
768   }
769 
770   /**
771    * Attempts to do an atomic load of many hfiles into a region.  If it fails,
772    * it returns a list of hfiles that need to be retried.  If it is successful
773    * it will return an empty list.
774    *
775    * NOTE: To maintain row atomicity guarantees, region server callable should
776    * succeed atomically and fails atomically.
777    *
778    * Protected for testing.
779    *
780    * @return empty list if success, list of items to retry on recoverable
781    * failure
782    */
783   protected List<LoadQueueItem> tryAtomicRegionLoad(final Connection conn,
784       final TableName tableName, final byte[] first, Collection<LoadQueueItem> lqis)
785   throws IOException {
786     final List<Pair<byte[], String>> famPaths =
787       new ArrayList<Pair<byte[], String>>(lqis.size());
788     for (LoadQueueItem lqi : lqis) {
789       famPaths.add(Pair.newPair(lqi.family, lqi.hfilePath.toString()));
790     }
791 
792     final RegionServerCallable<Boolean> svrCallable =
793         new RegionServerCallable<Boolean>(conn, tableName, first) {
794       @Override
795       public Boolean call(int callTimeout) throws Exception {
796         SecureBulkLoadClient secureClient = null;
797         boolean success = false;
798 
799         try {
800           LOG.debug("Going to connect to server " + getLocation() + " for row "
801               + Bytes.toStringBinary(getRow()) + " with hfile group " + famPaths);
802           byte[] regionName = getLocation().getRegionInfo().getRegionName();
803           if (!isSecureBulkLoadEndpointAvailable()) {
804             success = ProtobufUtil.bulkLoadHFile(getStub(), famPaths, regionName, assignSeqIds);
805           } else {
806             try (Table table = conn.getTable(getTableName())) {
807               secureClient = new SecureBulkLoadClient(table);
808               success = secureClient.bulkLoadHFiles(famPaths, fsDelegationToken.getUserToken(),
809                 bulkToken, getLocation().getRegionInfo().getStartKey());
810             }
811           }
812           return success;
813         } finally {
814           //Best effort copying of files that might not have been imported
815           //from the staging directory back to original location
816           //in user directory
817           if(secureClient != null && !success) {
818             FileSystem targetFs = FileSystem.get(getConf());
819             // Check to see if the source and target filesystems are the same
820             // If they are the same filesystem, we will try move the files back
821             // because previously we moved them to the staging directory.
822             if (FSHDFSUtils.isSameHdfs(getConf(), fs, targetFs)) {
823               for(Pair<byte[], String> el : famPaths) {
824                 Path hfileStagingPath = null;
825                 Path hfileOrigPath = new Path(el.getSecond());
826                 try {
827                   hfileStagingPath= new Path(secureClient.getStagingPath(bulkToken, el.getFirst()),
828                     hfileOrigPath.getName());
829                   if(targetFs.rename(hfileStagingPath, hfileOrigPath)) {
830                     LOG.debug("Moved back file " + hfileOrigPath + " from " +
831                         hfileStagingPath);
832                   } else if(targetFs.exists(hfileStagingPath)){
833                     LOG.debug("Unable to move back file " + hfileOrigPath + " from " +
834                         hfileStagingPath);
835                   }
836                 } catch(Exception ex) {
837                   LOG.debug("Unable to move back file " + hfileOrigPath + " from " +
838                       hfileStagingPath, ex);
839                 }
840               }
841             }
842           }
843         }
844       }
845     };
846 
847     try {
848       List<LoadQueueItem> toRetry = new ArrayList<LoadQueueItem>();
849       Configuration conf = getConf();
850       boolean success = RpcRetryingCallerFactory.instantiate(conf,
851           null).<Boolean> newCaller()
852           .callWithRetries(svrCallable, Integer.MAX_VALUE);
853       if (!success) {
854         LOG.warn("Attempt to bulk load region containing "
855             + Bytes.toStringBinary(first) + " into table "
856             + tableName  + " with files " + lqis
857             + " failed.  This is recoverable and they will be retried.");
858         toRetry.addAll(lqis); // return lqi's to retry
859       }
860       // success
861       return toRetry;
862     } catch (IOException e) {
863       LOG.error("Encountered unrecoverable error from region server, additional details: "
864           + svrCallable.getExceptionMessageAdditionalDetail(), e);
865       throw e;
866     }
867   }
868 
869   private boolean isSecureBulkLoadEndpointAvailable() {
870     String classes = getConf().get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
871     return classes.contains("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
872   }
873 
874   /**
875    * Split a storefile into a top and bottom half, maintaining
876    * the metadata, recreating bloom filters, etc.
877    */
878   static void splitStoreFile(
879       Configuration conf, Path inFile,
880       HColumnDescriptor familyDesc, byte[] splitKey,
881       Path bottomOut, Path topOut) throws IOException
882   {
883     // Open reader with no block cache, and not in-memory
884     Reference topReference = Reference.createTopReference(splitKey);
885     Reference bottomReference = Reference.createBottomReference(splitKey);
886 
887     copyHFileHalf(conf, inFile, topOut, topReference, familyDesc);
888     copyHFileHalf(conf, inFile, bottomOut, bottomReference, familyDesc);
889   }
890 
891   /**
892    * Copy half of an HFile into a new HFile.
893    */
894   private static void copyHFileHalf(
895       Configuration conf, Path inFile, Path outFile, Reference reference,
896       HColumnDescriptor familyDescriptor)
897   throws IOException {
898     FileSystem fs = inFile.getFileSystem(conf);
899     CacheConfig cacheConf = new CacheConfig(conf);
900     HalfStoreFileReader halfReader = null;
901     StoreFile.Writer halfWriter = null;
902     try {
903       halfReader = new HalfStoreFileReader(fs, inFile, cacheConf, reference, conf);
904       Map<byte[], byte[]> fileInfo = halfReader.loadFileInfo();
905 
906       int blocksize = familyDescriptor.getBlocksize();
907       Algorithm compression = familyDescriptor.getCompression();
908       BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
909       HFileContext hFileContext = new HFileContextBuilder()
910                                   .withCompression(compression)
911                                   .withChecksumType(HStore.getChecksumType(conf))
912                                   .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf))
913                                   .withBlockSize(blocksize)
914                                   .withDataBlockEncoding(familyDescriptor.getDataBlockEncoding())
915                                   .withIncludesTags(true)
916                                   .build();
917       halfWriter = new StoreFile.WriterBuilder(conf, cacheConf,
918           fs)
919               .withFilePath(outFile)
920               .withBloomType(bloomFilterType)
921               .withFileContext(hFileContext)
922               .build();
923       HFileScanner scanner = halfReader.getScanner(false, false, false);
924       scanner.seekTo();
925       do {
926         halfWriter.append(scanner.getKeyValue());
927       } while (scanner.next());
928 
929       for (Map.Entry<byte[],byte[]> entry : fileInfo.entrySet()) {
930         if (shouldCopyHFileMetaKey(entry.getKey())) {
931           halfWriter.appendFileInfo(entry.getKey(), entry.getValue());
932         }
933       }
934     } finally {
935       if (halfWriter != null) halfWriter.close();
936       if (halfReader != null) halfReader.close(cacheConf.shouldEvictOnClose());
937     }
938   }
939 
940   private static boolean shouldCopyHFileMetaKey(byte[] key) {
941     // skip encoding to keep hfile meta consistent with data block info, see HBASE-15085
942     if (Bytes.equals(key, HFileDataBlockEncoder.DATA_BLOCK_ENCODING)) {
943       return false;
944     }
945 
946     return !HFile.isReservedFileInfoKey(key);
947   }
948 
949   private boolean doesTableExist(TableName tableName) throws Exception {
950     return hbAdmin.tableExists(tableName);
951   }
952 
953   /*
954    * Infers region boundaries for a new table.
955    * Parameter:
956    *   bdryMap is a map between keys to an integer belonging to {+1, -1}
957    *     If a key is a start key of a file, then it maps to +1
958    *     If a key is an end key of a file, then it maps to -1
959    * Algo:
960    * 1) Poll on the keys in order:
961    *    a) Keep adding the mapped values to these keys (runningSum)
962    *    b) Each time runningSum reaches 0, add the start Key from when the runningSum had started to a boundary list.
963    * 2) Return the boundary list.
964    */
965   public static byte[][] inferBoundaries(TreeMap<byte[], Integer> bdryMap) {
966     ArrayList<byte[]> keysArray = new ArrayList<byte[]>();
967     int runningValue = 0;
968     byte[] currStartKey = null;
969     boolean firstBoundary = true;
970 
971     for (Map.Entry<byte[], Integer> item: bdryMap.entrySet()) {
972       if (runningValue == 0) currStartKey = item.getKey();
973       runningValue += item.getValue();
974       if (runningValue == 0) {
975         if (!firstBoundary) keysArray.add(currStartKey);
976         firstBoundary = false;
977       }
978     }
979 
980     return keysArray.toArray(new byte[0][0]);
981   }
982 
983   /*
984    * If the table is created for the first time, then "completebulkload" reads the files twice.
985    * More modifications necessary if we want to avoid doing it.
986    */
987   private void createTable(TableName tableName, String dirPath) throws Exception {
988     final Path hfofDir = new Path(dirPath);
989     final FileSystem fs = hfofDir.getFileSystem(getConf());
990 
991     // Add column families
992     // Build a set of keys
993     final HTableDescriptor htd = new HTableDescriptor(tableName);
994     final TreeMap<byte[], Integer> map = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
995     visitBulkHFiles(fs, hfofDir, new BulkHFileVisitor<HColumnDescriptor>() {
996       @Override
997       public HColumnDescriptor bulkFamily(final byte[] familyName) {
998         HColumnDescriptor hcd = new HColumnDescriptor(familyName);
999         htd.addFamily(hcd);
1000         return hcd;
1001       }
1002       @Override
1003       public void bulkHFile(final HColumnDescriptor hcd, final FileStatus hfileStatus)
1004           throws IOException {
1005         Path hfile = hfileStatus.getPath();
1006         HFile.Reader reader = HFile.createReader(fs, hfile,
1007             new CacheConfig(getConf()), getConf());
1008         try {
1009           if (hcd.getCompressionType() != reader.getFileContext().getCompression()) {
1010             hcd.setCompressionType(reader.getFileContext().getCompression());
1011             LOG.info("Setting compression " + hcd.getCompressionType().name() +
1012                      " for family " + hcd.toString());
1013           }
1014           reader.loadFileInfo();
1015           byte[] first = reader.getFirstRowKey();
1016           byte[] last  = reader.getLastRowKey();
1017 
1018           LOG.info("Trying to figure out region boundaries hfile=" + hfile +
1019             " first=" + Bytes.toStringBinary(first) +
1020             " last="  + Bytes.toStringBinary(last));
1021 
1022           // To eventually infer start key-end key boundaries
1023           Integer value = map.containsKey(first)? map.get(first):0;
1024           map.put(first, value+1);
1025 
1026           value = map.containsKey(last)? map.get(last):0;
1027           map.put(last, value-1);
1028         } finally {
1029           reader.close();
1030         }
1031       }
1032     });
1033 
1034     byte[][] keys = LoadIncrementalHFiles.inferBoundaries(map);
1035     this.hbAdmin.createTable(htd,keys);
1036 
1037     LOG.info("Table "+ tableName +" is available!!");
1038   }
1039 
1040   @Override
1041   public int run(String[] args) throws Exception {
1042     if (args.length != 2) {
1043       usage();
1044       return -1;
1045     }
1046 
1047     initialize();
1048 
1049     String dirPath = args[0];
1050     TableName tableName = TableName.valueOf(args[1]);
1051 
1052     boolean tableExists = this.doesTableExist(tableName);
1053     if (!tableExists) {
1054       if ("yes".equalsIgnoreCase(getConf().get(CREATE_TABLE_CONF_KEY, "yes"))) {
1055         this.createTable(tableName, dirPath);
1056       } else {
1057         String errorMsg = format("Table '%s' does not exist.", tableName);
1058         LOG.error(errorMsg);
1059         throw new TableNotFoundException(errorMsg);
1060       }
1061     }
1062     if (hbAdmin != null) {
1063       hbAdmin.close();
1064     }
1065     Path hfofDir = new Path(dirPath);
1066 
1067     try (Connection connection = ConnectionFactory.createConnection(getConf());
1068         HTable table = (HTable) connection.getTable(tableName);) {
1069       doBulkLoad(hfofDir, table);
1070     }
1071     return 0;
1072   }
1073 
1074   public static void main(String[] args) throws Exception {
1075     Configuration conf = HBaseConfiguration.create();
1076     int ret = ToolRunner.run(conf, new LoadIncrementalHFiles(), args);
1077     System.exit(ret);
1078   }
1079 
1080 }