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