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