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.LinkedList;
34  import java.util.List;
35  import java.util.Map;
36  import java.util.Map.Entry;
37  import java.util.Set;
38  import java.util.TreeMap;
39  import java.util.concurrent.Callable;
40  import java.util.concurrent.ExecutionException;
41  import java.util.concurrent.ExecutorService;
42  import java.util.concurrent.Future;
43  import java.util.concurrent.LinkedBlockingQueue;
44  import java.util.concurrent.ThreadPoolExecutor;
45  import java.util.concurrent.TimeUnit;
46  
47  import org.apache.commons.lang.mutable.MutableInt;
48  import org.apache.commons.logging.Log;
49  import org.apache.commons.logging.LogFactory;
50  import org.apache.hadoop.hbase.classification.InterfaceAudience;
51  import org.apache.hadoop.hbase.classification.InterfaceStability;
52  import org.apache.hadoop.conf.Configuration;
53  import org.apache.hadoop.conf.Configured;
54  import org.apache.hadoop.fs.permission.FsPermission;
55  import org.apache.hadoop.fs.FileStatus;
56  import org.apache.hadoop.fs.FileSystem;
57  import org.apache.hadoop.fs.FileUtil;
58  import org.apache.hadoop.fs.Path;
59  import org.apache.hadoop.hbase.HBaseConfiguration;
60  import org.apache.hadoop.hbase.HColumnDescriptor;
61  import org.apache.hadoop.hbase.HConstants;
62  import org.apache.hadoop.hbase.HTableDescriptor;
63  import org.apache.hadoop.hbase.KeyValue;
64  import org.apache.hadoop.hbase.KeyValueUtil;
65  import org.apache.hadoop.hbase.TableName;
66  import org.apache.hadoop.hbase.TableNotFoundException;
67  import org.apache.hadoop.hbase.client.Admin;
68  import org.apache.hadoop.hbase.client.HBaseAdmin;
69  import org.apache.hadoop.hbase.client.HConnection;
70  import org.apache.hadoop.hbase.client.HTable;
71  import org.apache.hadoop.hbase.client.RegionServerCallable;
72  import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
73  import org.apache.hadoop.hbase.client.Table;
74  import org.apache.hadoop.hbase.client.coprocessor.SecureBulkLoadClient;
75  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
76  import org.apache.hadoop.hbase.io.HFileLink;
77  import org.apache.hadoop.hbase.io.HalfStoreFileReader;
78  import org.apache.hadoop.hbase.io.Reference;
79  import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
80  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
81  import org.apache.hadoop.hbase.io.hfile.HFile;
82  import org.apache.hadoop.hbase.io.hfile.HFileContext;
83  import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
84  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
85  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
86  import org.apache.hadoop.hbase.regionserver.BloomType;
87  import org.apache.hadoop.hbase.regionserver.HStore;
88  import org.apache.hadoop.hbase.regionserver.StoreFile;
89  import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
90  import org.apache.hadoop.hbase.security.UserProvider;
91  import org.apache.hadoop.hbase.security.token.FsDelegationToken;
92  import org.apache.hadoop.hbase.util.Bytes;
93  import org.apache.hadoop.hbase.util.FSHDFSUtils;
94  import org.apache.hadoop.hbase.util.Pair;
95  import org.apache.hadoop.util.Tool;
96  import org.apache.hadoop.util.ToolRunner;
97  
98  import com.google.common.collect.HashMultimap;
99  import com.google.common.collect.Multimap;
100 import com.google.common.collect.Multimaps;
101 import com.google.common.util.concurrent.ThreadFactoryBuilder;
102 
103 import java.util.UUID;
104 
105 /**
106  * Tool to load the output of HFileOutputFormat into an existing table.
107  * @see #usage()
108  */
109 @InterfaceAudience.Public
110 @InterfaceStability.Stable
111 public class LoadIncrementalHFiles extends Configured implements Tool {
112   private static final Log LOG = LogFactory.getLog(LoadIncrementalHFiles.class);
113   private Admin hbAdmin;
114 
115   public static final String NAME = "completebulkload";
116   public static final String MAX_FILES_PER_REGION_PER_FAMILY
117     = "hbase.mapreduce.bulkload.max.hfiles.perRegion.perFamily";
118   private static final String ASSIGN_SEQ_IDS = "hbase.mapreduce.bulkload.assign.sequenceNumbers";
119   public final static String CREATE_TABLE_CONF_KEY = "create.table";
120 
121   private int maxFilesPerRegionPerFamily;
122   private boolean assignSeqIds;
123 
124   // Source filesystem
125   private FileSystem fs;
126   // Source delegation token
127   private FsDelegationToken fsDelegationToken;
128   private String bulkToken;
129   private UserProvider userProvider;
130 
131   private LoadIncrementalHFiles() {}
132 
133   public LoadIncrementalHFiles(Configuration conf) throws Exception {
134     super(conf);
135     initialize();
136   }
137 
138   private void initialize() throws Exception {
139     if (hbAdmin == null) {
140       // make a copy, just to be sure we're not overriding someone else's config
141       setConf(HBaseConfiguration.create(getConf()));
142       Configuration conf = getConf();
143       // disable blockcache for tool invocation, see HBASE-10500
144       conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0);
145       this.hbAdmin = new HBaseAdmin(conf);
146       this.userProvider = UserProvider.instantiate(conf);
147       this.fsDelegationToken = new FsDelegationToken(userProvider, "renewer");
148       assignSeqIds = conf.getBoolean(ASSIGN_SEQ_IDS, true);
149       maxFilesPerRegionPerFamily = conf.getInt(MAX_FILES_PER_REGION_PER_FAMILY, 32);
150     }
151   }
152 
153   private void usage() {
154     System.err.println("usage: " + NAME + " /path/to/hfileoutputformat-output tablename" + "\n -D"
155         + CREATE_TABLE_CONF_KEY + "=no - can be used to avoid creation of table by this tool\n"
156         + "  Note: if you set this to 'no', then the target table must already exist in HBase\n"
157         + "\n");
158   }
159 
160   /**
161    * Represents an HFile waiting to be loaded. An queue is used
162    * in this class in order to support the case where a region has
163    * split during the process of the load. When this happens,
164    * the HFile is split into two physical parts across the new
165    * region boundary, and each part is added back into the queue.
166    * The import process finishes when the queue is empty.
167    */
168   static class LoadQueueItem {
169     final byte[] family;
170     final Path hfilePath;
171 
172     public LoadQueueItem(byte[] family, Path hfilePath) {
173       this.family = family;
174       this.hfilePath = hfilePath;
175     }
176 
177     public String toString() {
178       return "family:"+ Bytes.toString(family) + " path:" + hfilePath.toString();
179     }
180   }
181 
182   /**
183    * Walk the given directory for all HFiles, and return a Queue
184    * containing all such files.
185    */
186   private void discoverLoadQueue(Deque<LoadQueueItem> ret, Path hfofDir)
187   throws IOException {
188     fs = hfofDir.getFileSystem(getConf());
189 
190     if (!fs.exists(hfofDir)) {
191       throw new FileNotFoundException("HFileOutputFormat dir " +
192           hfofDir + " not found");
193     }
194 
195     FileStatus[] familyDirStatuses = fs.listStatus(hfofDir);
196     if (familyDirStatuses == null) {
197       throw new FileNotFoundException("No families found in " + hfofDir);
198     }
199 
200     for (FileStatus stat : familyDirStatuses) {
201       if (!stat.isDirectory()) {
202         LOG.warn("Skipping non-directory " + stat.getPath());
203         continue;
204       }
205       Path familyDir = stat.getPath();
206       byte[] family = familyDir.getName().getBytes();
207       Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
208       for (Path hfile : hfiles) {
209         // Skip "_", reference, HFileLink
210         String fileName = hfile.getName();
211         if (fileName.startsWith("_")) continue;
212         if (StoreFileInfo.isReference(fileName)) {
213           LOG.warn("Skipping reference " + fileName);
214           continue;
215         }
216         if (HFileLink.isHFileLink(fileName)) {
217           LOG.warn("Skipping HFileLink " + fileName);
218           continue;
219         }
220         ret.add(new LoadQueueItem(family, hfile));
221       }
222     }
223   }
224 
225   /**
226    * Perform a bulk load of the given directory into the given
227    * pre-existing table.  This method is not threadsafe.
228    *
229    * @param hfofDir the directory that was provided as the output path
230    * of a job using HFileOutputFormat
231    * @param table the table to load into
232    * @throws TableNotFoundException if table does not yet exist
233    */
234   @SuppressWarnings("deprecation")
235   public void doBulkLoad(Path hfofDir, final HTable table)
236     throws TableNotFoundException, IOException
237   {
238     final HConnection conn = table.getConnection();
239 
240     if (!conn.isTableAvailable(table.getName())) {
241       throw new TableNotFoundException("Table " +
242           Bytes.toStringBinary(table.getTableName()) +
243           "is not currently available.");
244     }
245 
246     // initialize thread pools
247     int nrThreads = getConf().getInt("hbase.loadincremental.threads.max",
248       Runtime.getRuntime().availableProcessors());
249     ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
250     builder.setNameFormat("LoadIncrementalHFiles-%1$d");
251     ExecutorService pool = new ThreadPoolExecutor(nrThreads, nrThreads,
252         60, TimeUnit.SECONDS,
253         new LinkedBlockingQueue<Runnable>(),
254         builder.build());
255     ((ThreadPoolExecutor)pool).allowCoreThreadTimeOut(true);
256 
257     // LQI queue does not need to be threadsafe -- all operations on this queue
258     // happen in this thread
259     Deque<LoadQueueItem> queue = new LinkedList<LoadQueueItem>();
260     try {
261       discoverLoadQueue(queue, hfofDir);
262       // check whether there is invalid family name in HFiles to be bulkloaded
263       Collection<HColumnDescriptor> families = table.getTableDescriptor().getFamilies();
264       ArrayList<String> familyNames = new ArrayList<String>();
265       for (HColumnDescriptor family : families) {
266         familyNames.add(family.getNameAsString());
267       }
268       ArrayList<String> unmatchedFamilies = new ArrayList<String>();
269       for (LoadQueueItem lqi : queue) {
270         String familyNameInHFile = Bytes.toString(lqi.family);
271         if (!familyNames.contains(familyNameInHFile)) {
272           unmatchedFamilies.add(familyNameInHFile);
273         }
274       }
275       if (unmatchedFamilies.size() > 0) {
276         String msg =
277             "Unmatched family names found: unmatched family names in HFiles to be bulkloaded: "
278                 + unmatchedFamilies + "; valid family names of table "
279                 + Bytes.toString(table.getTableName()) + " are: " + familyNames;
280         LOG.error(msg);
281         throw new IOException(msg);
282       }
283       int count = 0;
284 
285       if (queue.isEmpty()) {
286         LOG.warn("Bulk load operation did not find any files to load in " +
287             "directory " + hfofDir.toUri() + ".  Does it contain files in " +
288             "subdirectories that correspond to column family names?");
289         return;
290       }
291 
292       //If using secure bulk load, get source delegation token, and
293       //prepare staging directory and token
294       // fs is the source filesystem
295       fsDelegationToken.acquireDelegationToken(fs);
296       if(isSecureBulkLoadEndpointAvailable()) {
297         bulkToken = new SecureBulkLoadClient(table).prepareBulkLoad(table.getName());
298       }
299 
300       // Assumes that region splits can happen while this occurs.
301       while (!queue.isEmpty()) {
302         // need to reload split keys each iteration.
303         final Pair<byte[][], byte[][]> startEndKeys = table.getStartEndKeys();
304         if (count != 0) {
305           LOG.info("Split occured while grouping HFiles, retry attempt " +
306               + count + " with " + queue.size() + " files remaining to group or split");
307         }
308 
309         int maxRetries = getConf().getInt("hbase.bulkload.retries.number", 10);
310         if (maxRetries != 0 && count >= maxRetries) {
311           throw new IOException("Retry attempted " + count +
312             " times without completing, bailing out");
313         }
314         count++;
315 
316         // Using ByteBuffer for byte[] equality semantics
317         Multimap<ByteBuffer, LoadQueueItem> regionGroups = groupOrSplitPhase(table,
318             pool, queue, startEndKeys);
319 
320         if (!checkHFilesCountPerRegionPerFamily(regionGroups)) {
321           // Error is logged inside checkHFilesCountPerRegionPerFamily.
322           throw new IOException("Trying to load more than " + maxFilesPerRegionPerFamily
323             + " hfiles to one family of one region");
324         }
325 
326         bulkLoadPhase(table, conn, pool, queue, regionGroups);
327 
328         // NOTE: The next iteration's split / group could happen in parallel to
329         // atomic bulkloads assuming that there are splits and no merges, and
330         // that we can atomically pull out the groups we want to retry.
331       }
332 
333     } finally {
334       fsDelegationToken.releaseDelegationToken();
335       if(bulkToken != null) {
336         new SecureBulkLoadClient(table).cleanupBulkLoad(bulkToken);
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 Table 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 Table 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 (!isSecureBulkLoadEndpointAvailable()) {
647             success = ProtobufUtil.bulkLoadHFile(getStub(), famPaths, regionName, assignSeqIds);
648           } else {
649             try (Table table = conn.getTable(getTableName())) {
650               secureClient = new SecureBulkLoadClient(table);
651               success = secureClient.bulkLoadHFiles(famPaths, fsDelegationToken.getUserToken(),
652                 bulkToken, getLocation().getRegionInfo().getStartKey());
653             }
654           }
655           return success;
656         } finally {
657           //Best effort copying of files that might not have been imported
658           //from the staging directory back to original location
659           //in user directory
660           if(secureClient != null && !success) {
661             FileSystem targetFs = FileSystem.get(getConf());
662             // Check to see if the source and target filesystems are the same
663             // If they are the same filesystem, we will try move the files back
664             // because previously we moved them to the staging directory.
665             if (FSHDFSUtils.isSameHdfs(getConf(), fs, targetFs)) {
666               for(Pair<byte[], String> el : famPaths) {
667                 Path hfileStagingPath = null;
668                 Path hfileOrigPath = new Path(el.getSecond());
669                 try {
670                   hfileStagingPath= new Path(secureClient.getStagingPath(bulkToken, el.getFirst()),
671                     hfileOrigPath.getName());
672                   if(targetFs.rename(hfileStagingPath, hfileOrigPath)) {
673                     LOG.debug("Moved back file " + hfileOrigPath + " from " +
674                         hfileStagingPath);
675                   } else if(targetFs.exists(hfileStagingPath)){
676                     LOG.debug("Unable to move back file " + hfileOrigPath + " from " +
677                         hfileStagingPath);
678                   }
679                 } catch(Exception ex) {
680                   LOG.debug("Unable to move back file " + hfileOrigPath + " from " +
681                       hfileStagingPath, ex);
682                 }
683               }
684             }
685           }
686         }
687       }
688     };
689 
690     try {
691       List<LoadQueueItem> toRetry = new ArrayList<LoadQueueItem>();
692       Configuration conf = getConf();
693       boolean success = RpcRetryingCallerFactory.instantiate(conf,
694           null).<Boolean> newCaller()
695           .callWithRetries(svrCallable, Integer.MAX_VALUE);
696       if (!success) {
697         LOG.warn("Attempt to bulk load region containing "
698             + Bytes.toStringBinary(first) + " into table "
699             + tableName  + " with files " + lqis
700             + " failed.  This is recoverable and they will be retried.");
701         toRetry.addAll(lqis); // return lqi's to retry
702       }
703       // success
704       return toRetry;
705     } catch (IOException e) {
706       LOG.error("Encountered unrecoverable error from region server, additional details: "
707           + svrCallable.getExceptionMessageAdditionalDetail(), e);
708       throw e;
709     }
710   }
711   
712   private boolean isSecureBulkLoadEndpointAvailable() {
713     String classes = getConf().get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
714     return classes.contains("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
715   }
716 
717   /**
718    * Split a storefile into a top and bottom half, maintaining
719    * the metadata, recreating bloom filters, etc.
720    */
721   static void splitStoreFile(
722       Configuration conf, Path inFile,
723       HColumnDescriptor familyDesc, byte[] splitKey,
724       Path bottomOut, Path topOut) throws IOException
725   {
726     // Open reader with no block cache, and not in-memory
727     Reference topReference = Reference.createTopReference(splitKey);
728     Reference bottomReference = Reference.createBottomReference(splitKey);
729 
730     copyHFileHalf(conf, inFile, topOut, topReference, familyDesc);
731     copyHFileHalf(conf, inFile, bottomOut, bottomReference, familyDesc);
732   }
733 
734   /**
735    * Copy half of an HFile into a new HFile.
736    */
737   private static void copyHFileHalf(
738       Configuration conf, Path inFile, Path outFile, Reference reference,
739       HColumnDescriptor familyDescriptor)
740   throws IOException {
741     FileSystem fs = inFile.getFileSystem(conf);
742     CacheConfig cacheConf = new CacheConfig(conf);
743     HalfStoreFileReader halfReader = null;
744     StoreFile.Writer halfWriter = null;
745     try {
746       halfReader = new HalfStoreFileReader(fs, inFile, cacheConf, reference, conf);
747       Map<byte[], byte[]> fileInfo = halfReader.loadFileInfo();
748 
749       int blocksize = familyDescriptor.getBlocksize();
750       Algorithm compression = familyDescriptor.getCompression();
751       BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
752       HFileContext hFileContext = new HFileContextBuilder()
753                                   .withCompression(compression)
754                                   .withChecksumType(HStore.getChecksumType(conf))
755                                   .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf))
756                                   .withBlockSize(blocksize)
757                                   .withDataBlockEncoding(familyDescriptor.getDataBlockEncoding())
758                                   .build();
759       halfWriter = new StoreFile.WriterBuilder(conf, cacheConf,
760           fs)
761               .withFilePath(outFile)
762               .withBloomType(bloomFilterType)
763               .withFileContext(hFileContext)
764               .build();
765       HFileScanner scanner = halfReader.getScanner(false, false, false);
766       scanner.seekTo();
767       do {
768         KeyValue kv = KeyValueUtil.ensureKeyValue(scanner.getKeyValue());
769         halfWriter.append(kv);
770       } while (scanner.next());
771 
772       for (Map.Entry<byte[],byte[]> entry : fileInfo.entrySet()) {
773         if (shouldCopyHFileMetaKey(entry.getKey())) {
774           halfWriter.appendFileInfo(entry.getKey(), entry.getValue());
775         }
776       }
777     } finally {
778       if (halfWriter != null) halfWriter.close();
779       if (halfReader != null) halfReader.close(cacheConf.shouldEvictOnClose());
780     }
781   }
782 
783   private static boolean shouldCopyHFileMetaKey(byte[] key) {
784     return !HFile.isReservedFileInfoKey(key);
785   }
786 
787   private boolean doesTableExist(TableName tableName) throws Exception {
788     return hbAdmin.tableExists(tableName);
789   }
790 
791   /*
792    * Infers region boundaries for a new table.
793    * Parameter:
794    *   bdryMap is a map between keys to an integer belonging to {+1, -1}
795    *     If a key is a start key of a file, then it maps to +1
796    *     If a key is an end key of a file, then it maps to -1
797    * Algo:
798    * 1) Poll on the keys in order:
799    *    a) Keep adding the mapped values to these keys (runningSum)
800    *    b) Each time runningSum reaches 0, add the start Key from when the runningSum had started to a boundary list.
801    * 2) Return the boundary list.
802    */
803   public static byte[][] inferBoundaries(TreeMap<byte[], Integer> bdryMap) {
804     ArrayList<byte[]> keysArray = new ArrayList<byte[]>();
805     int runningValue = 0;
806     byte[] currStartKey = null;
807     boolean firstBoundary = true;
808 
809     for (Map.Entry<byte[], Integer> item: bdryMap.entrySet()) {
810       if (runningValue == 0) currStartKey = item.getKey();
811       runningValue += item.getValue();
812       if (runningValue == 0) {
813         if (!firstBoundary) keysArray.add(currStartKey);
814         firstBoundary = false;
815       }
816     }
817 
818     return keysArray.toArray(new byte[0][0]);
819   }
820 
821   /*
822    * If the table is created for the first time, then "completebulkload" reads the files twice.
823    * More modifications necessary if we want to avoid doing it.
824    */
825   private void createTable(TableName tableName, String dirPath) throws Exception {
826     Path hfofDir = new Path(dirPath);
827     FileSystem fs = hfofDir.getFileSystem(getConf());
828 
829     if (!fs.exists(hfofDir)) {
830       throw new FileNotFoundException("HFileOutputFormat dir " +
831           hfofDir + " not found");
832     }
833 
834     FileStatus[] familyDirStatuses = fs.listStatus(hfofDir);
835     if (familyDirStatuses == null) {
836       throw new FileNotFoundException("No families found in " + hfofDir);
837     }
838 
839     HTableDescriptor htd = new HTableDescriptor(tableName);
840     HColumnDescriptor hcd;
841 
842     // Add column families
843     // Build a set of keys
844     byte[][] keys;
845     TreeMap<byte[], Integer> map = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
846 
847     for (FileStatus stat : familyDirStatuses) {
848       if (!stat.isDirectory()) {
849         LOG.warn("Skipping non-directory " + stat.getPath());
850         continue;
851       }
852       Path familyDir = stat.getPath();
853       byte[] family = familyDir.getName().getBytes();
854 
855       hcd = new HColumnDescriptor(family);
856       htd.addFamily(hcd);
857 
858       Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
859       for (Path hfile : hfiles) {
860         String fileName = hfile.getName();
861         if (fileName.startsWith("_") || StoreFileInfo.isReference(fileName)
862             || HFileLink.isHFileLink(fileName)) continue;
863         HFile.Reader reader = HFile.createReader(fs, hfile,
864             new CacheConfig(getConf()), getConf());
865         final byte[] first, last;
866         try {
867           if (hcd.getCompressionType() != reader.getFileContext().getCompression()) {
868             hcd.setCompressionType(reader.getFileContext().getCompression());
869             LOG.info("Setting compression " + hcd.getCompressionType().name() +
870                      " for family " + hcd.toString());
871           }
872           reader.loadFileInfo();
873           first = reader.getFirstRowKey();
874           last =  reader.getLastRowKey();
875 
876           LOG.info("Trying to figure out region boundaries hfile=" + hfile +
877             " first=" + Bytes.toStringBinary(first) +
878             " last="  + Bytes.toStringBinary(last));
879 
880           // To eventually infer start key-end key boundaries
881           Integer value = map.containsKey(first)? map.get(first):0;
882           map.put(first, value+1);
883 
884           value = map.containsKey(last)? map.get(last):0;
885           map.put(last, value-1);
886         }  finally {
887           reader.close();
888         }
889       }
890     }
891 
892     keys = LoadIncrementalHFiles.inferBoundaries(map);
893     this.hbAdmin.createTable(htd,keys);
894 
895     LOG.info("Table "+ tableName +" is available!!");
896   }
897 
898   @Override
899   public int run(String[] args) throws Exception {
900     if (args.length != 2) {
901       usage();
902       return -1;
903     }
904 
905     initialize();
906 
907     String dirPath = args[0];
908     TableName tableName = TableName.valueOf(args[1]);
909 
910     boolean tableExists = this.doesTableExist(tableName);
911     if (!tableExists) {
912       if ("yes".equalsIgnoreCase(getConf().get(CREATE_TABLE_CONF_KEY, "yes"))) {
913         this.createTable(tableName, dirPath);
914       } else {
915         String errorMsg = format("Table '%s' does not exist.", tableName);
916         LOG.error(errorMsg);
917         throw new TableNotFoundException(errorMsg);
918       }
919     }
920 
921     Path hfofDir = new Path(dirPath);
922     HTable table = new HTable(getConf(), tableName);
923 
924     doBulkLoad(hfofDir, table);
925     return 0;
926   }
927 
928   public static void main(String[] args) throws Exception {
929     Configuration conf = HBaseConfiguration.create();
930     int ret = ToolRunner.run(conf, new LoadIncrementalHFiles(), args);
931     System.exit(ret);
932   }
933 
934 }