View Javadoc

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