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