View Javadoc

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