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