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