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