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