View Javadoc

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