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