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