001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.tool;
019
020import static java.lang.String.format;
021
022import java.io.FileNotFoundException;
023import java.io.IOException;
024import java.io.InterruptedIOException;
025import java.nio.ByteBuffer;
026import java.util.ArrayDeque;
027import java.util.ArrayList;
028import java.util.Arrays;
029import java.util.Collection;
030import java.util.Collections;
031import java.util.Deque;
032import java.util.HashMap;
033import java.util.HashSet;
034import java.util.List;
035import java.util.Map;
036import java.util.Map.Entry;
037import java.util.Optional;
038import java.util.Set;
039import java.util.SortedMap;
040import java.util.TreeMap;
041import java.util.UUID;
042import java.util.concurrent.Callable;
043import java.util.concurrent.ExecutionException;
044import java.util.concurrent.ExecutorService;
045import java.util.concurrent.Future;
046import java.util.concurrent.LinkedBlockingQueue;
047import java.util.concurrent.ThreadPoolExecutor;
048import java.util.concurrent.TimeUnit;
049import java.util.concurrent.atomic.AtomicInteger;
050import java.util.stream.Collectors;
051import org.apache.commons.lang3.mutable.MutableInt;
052import org.apache.hadoop.conf.Configuration;
053import org.apache.hadoop.conf.Configured;
054import org.apache.hadoop.fs.FileStatus;
055import org.apache.hadoop.fs.FileSystem;
056import org.apache.hadoop.fs.Path;
057import org.apache.hadoop.fs.permission.FsPermission;
058import org.apache.hadoop.hbase.HBaseConfiguration;
059import org.apache.hadoop.hbase.HConstants;
060import org.apache.hadoop.hbase.TableName;
061import org.apache.hadoop.hbase.TableNotFoundException;
062import org.apache.hadoop.hbase.client.Admin;
063import org.apache.hadoop.hbase.client.ClientServiceCallable;
064import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
065import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
066import org.apache.hadoop.hbase.client.Connection;
067import org.apache.hadoop.hbase.client.ConnectionFactory;
068import org.apache.hadoop.hbase.client.RegionLocator;
069import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
070import org.apache.hadoop.hbase.client.SecureBulkLoadClient;
071import org.apache.hadoop.hbase.client.Table;
072import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
073import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
074import org.apache.hadoop.hbase.io.HFileLink;
075import org.apache.hadoop.hbase.io.HalfStoreFileReader;
076import org.apache.hadoop.hbase.io.Reference;
077import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
078import org.apache.hadoop.hbase.io.hfile.CacheConfig;
079import org.apache.hadoop.hbase.io.hfile.HFile;
080import org.apache.hadoop.hbase.io.hfile.HFileContext;
081import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
082import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
083import org.apache.hadoop.hbase.io.hfile.HFileInfo;
084import org.apache.hadoop.hbase.io.hfile.HFileScanner;
085import org.apache.hadoop.hbase.io.hfile.ReaderContext;
086import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder;
087import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
088import org.apache.hadoop.hbase.regionserver.BloomType;
089import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
090import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
091import org.apache.hadoop.hbase.regionserver.StoreUtils;
092import org.apache.hadoop.hbase.security.UserProvider;
093import org.apache.hadoop.hbase.security.token.FsDelegationToken;
094import org.apache.hadoop.hbase.util.Bytes;
095import org.apache.hadoop.hbase.util.FSUtils;
096import org.apache.hadoop.hbase.util.FSVisitor;
097import org.apache.hadoop.hbase.util.Pair;
098import org.apache.hadoop.util.Tool;
099import org.apache.hadoop.util.ToolRunner;
100import org.apache.yetus.audience.InterfaceAudience;
101import org.slf4j.Logger;
102import org.slf4j.LoggerFactory;
103
104import org.apache.hbase.thirdparty.com.google.common.collect.HashMultimap;
105import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
106import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
107import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
108import org.apache.hbase.thirdparty.com.google.common.collect.Multimaps;
109import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
110
111/**
112 * Tool to load the output of HFileOutputFormat into an existing table.
113 * <p/>
114 * Notice that, by default, this class should be kept till 4.0.0, but as this is a bad practice that
115 * we expose an implementation class instead of an interface, we want to fix it ASAP. That's why we
116 * will remove this class completely in 3.0.0. Please change your code to use
117 * {@link BulkLoadHFiles}.
118 * @deprecated since 2.2.0, will be removed in 3.0.0. Use {@link BulkLoadHFiles} instead. Please
119 *             rewrite your code if you rely on methods other than the {@link #run(Map, TableName)}
120 *             and {@link #run(String, TableName)}, as all the methods other than them will be
121 *             removed with no replacement.
122 */
123@Deprecated
124@InterfaceAudience.Public
125public class LoadIncrementalHFiles extends Configured implements Tool {
126
127  private static final Logger LOG = LoggerFactory.getLogger(LoadIncrementalHFiles.class);
128
129  /**
130   * @deprecated since 2.2.0, will be removed in 3.0.0, with no replacement. End user should not
131   *             depend on this value.
132   */
133  @Deprecated
134  public static final String NAME = BulkLoadHFilesTool.NAME;
135  static final String RETRY_ON_IO_EXCEPTION = BulkLoadHFiles.RETRY_ON_IO_EXCEPTION;
136  public static final String MAX_FILES_PER_REGION_PER_FAMILY =
137    BulkLoadHFiles.MAX_FILES_PER_REGION_PER_FAMILY;
138  private static final String ASSIGN_SEQ_IDS = BulkLoadHFiles.ASSIGN_SEQ_IDS;
139  public final static String CREATE_TABLE_CONF_KEY = BulkLoadHFiles.CREATE_TABLE_CONF_KEY;
140  public final static String IGNORE_UNMATCHED_CF_CONF_KEY =
141    BulkLoadHFiles.IGNORE_UNMATCHED_CF_CONF_KEY;
142  public final static String ALWAYS_COPY_FILES = BulkLoadHFiles.ALWAYS_COPY_FILES;
143
144  public static final String FAIL_IF_NEED_SPLIT_HFILE =
145    "hbase.loadincremental.fail.if.need.split.hfile";
146
147  // We use a '.' prefix which is ignored when walking directory trees
148  // above. It is invalid family name.
149  static final String TMP_DIR = ".tmp";
150
151  private int maxFilesPerRegionPerFamily;
152  private boolean assignSeqIds;
153  private boolean bulkLoadByFamily;
154
155  // Source delegation token
156  private FsDelegationToken fsDelegationToken;
157  private UserProvider userProvider;
158  private int nrThreads;
159  private AtomicInteger numRetries;
160  private RpcControllerFactory rpcControllerFactory;
161
162  private String bulkToken;
163
164  private List<String> clusterIds = new ArrayList<>();
165
166  private boolean replicate = true;
167
168  private boolean failIfNeedSplitHFile = false;
169
170  /**
171   * Represents an HFile waiting to be loaded. An queue is used in this class in order to support
172   * the case where a region has split during the process of the load. When this happens, the HFile
173   * is split into two physical parts across the new region boundary, and each part is added back
174   * into the queue. The import process finishes when the queue is empty.
175   * @deprecated since 2.2.0 and will be removed in 3.0.0. Use {@link BulkLoadHFiles} instead.
176   * @see BulkLoadHFiles
177   * @see <a href="https://issues.apache.org/jira/browse/HBASE-21782">HBASE-21782</a>
178   */
179  @InterfaceAudience.Public
180  @Deprecated
181  public static class LoadQueueItem extends BulkLoadHFiles.LoadQueueItem {
182
183    public LoadQueueItem(byte[] family, Path hfilePath) {
184      super(family, hfilePath);
185    }
186  }
187
188  public LoadIncrementalHFiles(Configuration conf) {
189    // make a copy, just to be sure we're not overriding someone else's config
190    super(HBaseConfiguration.create(conf));
191    initialize();
192  }
193
194  public void initialize() {
195    Configuration conf = getConf();
196    // disable blockcache for tool invocation, see HBASE-10500
197    conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0);
198    userProvider = UserProvider.instantiate(conf);
199    fsDelegationToken = new FsDelegationToken(userProvider, "renewer");
200    assignSeqIds = conf.getBoolean(ASSIGN_SEQ_IDS, true);
201    maxFilesPerRegionPerFamily = conf.getInt(MAX_FILES_PER_REGION_PER_FAMILY, 32);
202    bulkLoadByFamily = conf.getBoolean(BulkLoadHFiles.BULK_LOAD_HFILES_BY_FAMILY, false);
203    failIfNeedSplitHFile = conf.getBoolean(FAIL_IF_NEED_SPLIT_HFILE, false);
204    nrThreads =
205      conf.getInt("hbase.loadincremental.threads.max", Runtime.getRuntime().availableProcessors());
206    numRetries = new AtomicInteger(0);
207    rpcControllerFactory = new RpcControllerFactory(conf);
208  }
209
210  private void usage() {
211    System.err.println("Usage: " + "bin/hbase completebulkload [OPTIONS] "
212      + "</PATH/TO/HFILEOUTPUTFORMAT-OUTPUT> <TABLENAME>\n"
213      + "Loads directory of hfiles -- a region dir or product of HFileOutputFormat -- "
214      + "into an hbase table.\n" + "OPTIONS (for other -D options, see source code):\n" + " -D"
215      + CREATE_TABLE_CONF_KEY + "=no whether to create table; when 'no', target "
216      + "table must exist.\n" + " -D" + IGNORE_UNMATCHED_CF_CONF_KEY
217      + "=yes to ignore unmatched column families.\n"
218      + " -loadTable for when directory of files to load has a depth of 3; target table must "
219      + "exist;\n" + " must be last of the options on command line.\n"
220      + "See http://hbase.apache.org/book.html#arch.bulk.load.complete.strays for "
221      + "documentation.\n");
222  }
223
224  /**
225   * Prepare a collection of {@link LoadQueueItem} from list of source hfiles contained in the
226   * passed directory and validates whether the prepared queue has all the valid table column
227   * families in it.
228   * @param hfilesDir     directory containing list of hfiles to be loaded into the table
229   * @param table         table to which hfiles should be loaded
230   * @param queue         queue which needs to be loaded into the table
231   * @param validateHFile if true hfiles will be validated for its format
232   * @throws IOException If any I/O or network error occurred
233   */
234  public void prepareHFileQueue(Path hfilesDir, Table table, Deque<LoadQueueItem> queue,
235    boolean validateHFile) throws IOException {
236    prepareHFileQueue(hfilesDir, table, queue, validateHFile, false);
237  }
238
239  /**
240   * Prepare a collection of {@link LoadQueueItem} from list of source hfiles contained in the
241   * passed directory and validates whether the prepared queue has all the valid table column
242   * families in it.
243   * @param hfilesDir     directory containing list of hfiles to be loaded into the table
244   * @param table         table to which hfiles should be loaded
245   * @param queue         queue which needs to be loaded into the table
246   * @param validateHFile if true hfiles will be validated for its format
247   * @param silence       true to ignore unmatched column families
248   * @throws IOException If any I/O or network error occurred
249   */
250  public void prepareHFileQueue(Path hfilesDir, Table table, Deque<LoadQueueItem> queue,
251    boolean validateHFile, boolean silence) throws IOException {
252    discoverLoadQueue(queue, hfilesDir, validateHFile);
253    validateFamiliesInHFiles(table, queue, silence);
254  }
255
256  /**
257   * Prepare a collection of {@link LoadQueueItem} from list of source hfiles contained in the
258   * passed directory and validates whether the prepared queue has all the valid table column
259   * families in it.
260   * @param map     map of family to List of hfiles
261   * @param table   table to which hfiles should be loaded
262   * @param queue   queue which needs to be loaded into the table
263   * @param silence true to ignore unmatched column families
264   * @throws IOException If any I/O or network error occurred
265   */
266  public void prepareHFileQueue(Map<byte[], List<Path>> map, Table table,
267    Deque<LoadQueueItem> queue, boolean silence) throws IOException {
268    populateLoadQueue(queue, map);
269    validateFamiliesInHFiles(table, queue, silence);
270  }
271
272  /**
273   * Perform a bulk load of the given directory into the given pre-existing table. This method is
274   * not threadsafe.
275   * @param hfofDir       the directory that was provided as the output path of a job using
276   *                      HFileOutputFormat
277   * @param admin         the Admin
278   * @param table         the table to load into
279   * @param regionLocator region locator
280   * @throws TableNotFoundException if table does not yet exist
281   */
282  public Map<LoadQueueItem, ByteBuffer> doBulkLoad(Path hfofDir, final Admin admin, Table table,
283    RegionLocator regionLocator) throws TableNotFoundException, IOException {
284    return doBulkLoad(hfofDir, admin, table, regionLocator, false, false);
285  }
286
287  /**
288   * Perform a bulk load of the given directory into the given pre-existing table. This method is
289   * not threadsafe.
290   * @param map           map of family to List of hfiles
291   * @param admin         the Admin
292   * @param table         the table to load into
293   * @param regionLocator region locator
294   * @param silence       true to ignore unmatched column families
295   * @param copyFile      always copy hfiles if true
296   * @throws TableNotFoundException if table does not yet exist
297   */
298  public Map<LoadQueueItem, ByteBuffer> doBulkLoad(Map<byte[], List<Path>> map, final Admin admin,
299    Table table, RegionLocator regionLocator, boolean silence, boolean copyFile)
300    throws TableNotFoundException, IOException {
301    if (!admin.isTableAvailable(regionLocator.getName())) {
302      throw new TableNotFoundException("Table " + table.getName() + " is not currently available.");
303    }
304    // LQI queue does not need to be threadsafe -- all operations on this queue
305    // happen in this thread
306    Deque<LoadQueueItem> queue = new ArrayDeque<>();
307    ExecutorService pool = null;
308    SecureBulkLoadClient secureClient = null;
309    try {
310      prepareHFileQueue(map, table, queue, silence);
311      if (queue.isEmpty()) {
312        LOG.warn("Bulk load operation did not get any files to load");
313        return Collections.emptyMap();
314      }
315      pool = createExecutorService();
316      secureClient = new SecureBulkLoadClient(table.getConfiguration(), table);
317      return performBulkLoad(admin, table, regionLocator, queue, pool, secureClient, copyFile);
318    } finally {
319      cleanup(admin, queue, pool, secureClient);
320    }
321  }
322
323  /**
324   * Perform a bulk load of the given directory into the given pre-existing table. This method is
325   * not threadsafe.
326   * @param hfofDir       the directory that was provided as the output path of a job using
327   *                      HFileOutputFormat
328   * @param admin         the Admin
329   * @param table         the table to load into
330   * @param regionLocator region locator
331   * @param silence       true to ignore unmatched column families
332   * @param copyFile      always copy hfiles if true
333   * @throws TableNotFoundException if table does not yet exist
334   */
335  public Map<LoadQueueItem, ByteBuffer> doBulkLoad(Path hfofDir, final Admin admin, Table table,
336    RegionLocator regionLocator, boolean silence, boolean copyFile)
337    throws TableNotFoundException, IOException {
338    if (!admin.isTableAvailable(regionLocator.getName())) {
339      throw new TableNotFoundException("Table " + table.getName() + " is not currently available.");
340    }
341
342    /*
343     * Checking hfile format is a time-consuming operation, we should have an option to skip this
344     * step when bulkloading millions of HFiles. See HBASE-13985.
345     */
346    boolean validateHFile = getConf().getBoolean("hbase.loadincremental.validate.hfile", true);
347    if (!validateHFile) {
348      LOG.warn("You are skipping HFiles validation, it might cause some data loss if files "
349        + "are not correct. If you fail to read data from your table after using this "
350        + "option, consider removing the files and bulkload again without this option. "
351        + "See HBASE-13985");
352    }
353    // LQI queue does not need to be threadsafe -- all operations on this queue
354    // happen in this thread
355    Deque<LoadQueueItem> queue = new ArrayDeque<>();
356    ExecutorService pool = null;
357    SecureBulkLoadClient secureClient = null;
358    try {
359      prepareHFileQueue(hfofDir, table, queue, validateHFile, silence);
360
361      if (queue.isEmpty()) {
362        LOG.warn(
363          "Bulk load operation did not find any files to load in directory {}. "
364            + "Does it contain files in subdirectories that correspond to column family names?",
365          (hfofDir != null ? hfofDir.toUri().toString() : ""));
366        return Collections.emptyMap();
367      }
368      pool = createExecutorService();
369      secureClient = new SecureBulkLoadClient(table.getConfiguration(), table);
370      return performBulkLoad(admin, table, regionLocator, queue, pool, secureClient, copyFile);
371    } finally {
372      cleanup(admin, queue, pool, secureClient);
373    }
374  }
375
376  /**
377   * Used by the replication sink to load the hfiles from the source cluster. It does the following,
378   * <ol>
379   * <li>LoadIncrementalHFiles#groupOrSplitPhase(Table, ExecutorService, Deque, Pair)}</li>
380   * <li>LoadIncrementalHFiles#bulkLoadPhase(Table, Connection, ExecutorService, Deque, Multimap)
381   * </li>
382   * </ol>
383   * @param table        Table to which these hfiles should be loaded to
384   * @param conn         Connection to use
385   * @param queue        {@link LoadQueueItem} has hfiles yet to be loaded
386   * @param startEndKeys starting and ending row keys of the region
387   */
388  public void loadHFileQueue(Table table, Connection conn, Deque<LoadQueueItem> queue,
389    Pair<byte[][], byte[][]> startEndKeys) throws IOException {
390    loadHFileQueue(table, conn, queue, startEndKeys, false);
391  }
392
393  /**
394   * Used by the replication sink to load the hfiles from the source cluster. It does the following,
395   * <ol>
396   * <li>LoadIncrementalHFiles#groupOrSplitPhase(Table, ExecutorService, Deque, Pair)}</li>
397   * <li>LoadIncrementalHFiles#bulkLoadPhase(Table, Connection, ExecutorService, Deque, Multimap)
398   * </li>
399   * </ol>
400   * @param table        Table to which these hfiles should be loaded to
401   * @param conn         Connection to use
402   * @param queue        {@link LoadQueueItem} has hfiles yet to be loaded
403   * @param startEndKeys starting and ending row keys of the region
404   */
405  public void loadHFileQueue(Table table, Connection conn, Deque<LoadQueueItem> queue,
406    Pair<byte[][], byte[][]> startEndKeys, boolean copyFile) throws IOException {
407    ExecutorService pool = null;
408    try {
409      pool = createExecutorService();
410      Multimap<ByteBuffer, LoadQueueItem> regionGroups =
411        groupOrSplitPhase(table, pool, queue, startEndKeys).getFirst();
412      bulkLoadPhase(table, conn, pool, queue, regionGroups, copyFile, null);
413    } finally {
414      if (pool != null) {
415        pool.shutdown();
416      }
417    }
418  }
419
420  private Map<LoadQueueItem, ByteBuffer> performBulkLoad(Admin admin, Table table,
421    RegionLocator regionLocator, Deque<LoadQueueItem> queue, ExecutorService pool,
422    SecureBulkLoadClient secureClient, boolean copyFile) throws IOException {
423    int count = 0;
424
425    if (isSecureBulkLoadEndpointAvailable()) {
426      LOG.warn("SecureBulkLoadEndpoint is deprecated. It will be removed in future releases.");
427      LOG.warn("Secure bulk load has been integrated into HBase core.");
428    }
429
430    fsDelegationToken.acquireDelegationToken(queue.peek().getFilePath().getFileSystem(getConf()));
431    bulkToken = secureClient.prepareBulkLoad(admin.getConnection());
432    Pair<Multimap<ByteBuffer, LoadQueueItem>, Set<String>> pair = null;
433
434    Map<LoadQueueItem, ByteBuffer> item2RegionMap = new HashMap<>();
435    // Assumes that region splits can happen while this occurs.
436    while (!queue.isEmpty()) {
437      // need to reload split keys each iteration.
438      final Pair<byte[][], byte[][]> startEndKeys = regionLocator.getStartEndKeys();
439      if (count != 0) {
440        LOG.info("Split occurred while grouping HFiles, retry attempt " + count + " with "
441          + queue.size() + " files remaining to group or split");
442      }
443
444      int maxRetries = getConf().getInt(HConstants.BULKLOAD_MAX_RETRIES_NUMBER, 10);
445      maxRetries = Math.max(maxRetries, startEndKeys.getFirst().length + 1);
446      if (maxRetries != 0 && count >= maxRetries) {
447        throw new IOException(
448          "Retry attempted " + count + " times without completing, bailing out");
449      }
450      count++;
451
452      // Using ByteBuffer for byte[] equality semantics
453      pair = groupOrSplitPhase(table, pool, queue, startEndKeys);
454      Multimap<ByteBuffer, LoadQueueItem> regionGroups = pair.getFirst();
455
456      if (!checkHFilesCountPerRegionPerFamily(regionGroups)) {
457        // Error is logged inside checkHFilesCountPerRegionPerFamily.
458        throw new IOException("Trying to load more than " + maxFilesPerRegionPerFamily
459          + " hfiles to one family of one region");
460      }
461
462      bulkLoadPhase(table, admin.getConnection(), pool, queue, regionGroups, copyFile,
463        item2RegionMap);
464
465      // NOTE: The next iteration's split / group could happen in parallel to
466      // atomic bulkloads assuming that there are splits and no merges, and
467      // that we can atomically pull out the groups we want to retry.
468    }
469
470    if (!queue.isEmpty()) {
471      throw new RuntimeException(
472        "Bulk load aborted with some files not yet loaded." + "Please check log for more details.");
473    }
474    return item2RegionMap;
475  }
476
477  private Map<byte[], Collection<LoadQueueItem>>
478    groupByFamilies(Collection<LoadQueueItem> itemsInRegion) {
479    Map<byte[], Collection<LoadQueueItem>> families2Queue = new TreeMap<>(Bytes.BYTES_COMPARATOR);
480    itemsInRegion.forEach(item -> families2Queue
481      .computeIfAbsent(item.getFamily(), queue -> new ArrayList<>()).add(item));
482    return families2Queue;
483  }
484
485  /**
486   * This takes the LQI's grouped by likely regions and attempts to bulk load them. Any failures are
487   * re-queued for another pass with the groupOrSplitPhase.
488   * <p>
489   * protected for testing.
490   */
491  @InterfaceAudience.Private
492  protected void bulkLoadPhase(Table table, Connection conn, ExecutorService pool,
493    Deque<LoadQueueItem> queue, Multimap<ByteBuffer, LoadQueueItem> regionGroups, boolean copyFile,
494    Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
495    // atomically bulk load the groups.
496    Set<Future<List<LoadQueueItem>>> loadingFutures = new HashSet<>();
497    for (Entry<ByteBuffer, ? extends Collection<LoadQueueItem>> e : regionGroups.asMap()
498      .entrySet()) {
499      byte[] first = e.getKey().array();
500      Collection<LoadQueueItem> lqis = e.getValue();
501      if (item2RegionMap != null) {
502        for (LoadQueueItem lqi : lqis) {
503          item2RegionMap.put(lqi, e.getKey());
504        }
505      }
506      if (bulkLoadByFamily) {
507        groupByFamilies(lqis).values().forEach(familyQueue -> loadingFutures.add(pool
508          .submit(() -> tryAtomicRegionLoad(conn, table.getName(), first, familyQueue, copyFile))));
509      } else {
510        loadingFutures.add(
511          pool.submit(() -> tryAtomicRegionLoad(conn, table.getName(), first, lqis, copyFile)));
512      }
513    }
514
515    // get all the results.
516    for (Future<List<LoadQueueItem>> future : loadingFutures) {
517      try {
518        List<LoadQueueItem> toRetry = future.get();
519
520        if (item2RegionMap != null) {
521          for (LoadQueueItem lqi : toRetry) {
522            item2RegionMap.remove(lqi);
523          }
524        }
525        // LQIs that are requeued to be regrouped.
526        queue.addAll(toRetry);
527
528      } catch (ExecutionException e1) {
529        Throwable t = e1.getCause();
530        if (t instanceof IOException) {
531          // At this point something unrecoverable has happened.
532          // TODO Implement bulk load recovery
533          throw new IOException("BulkLoad encountered an unrecoverable problem", t);
534        }
535        LOG.error("Unexpected execution exception during bulk load", e1);
536        throw new IllegalStateException(t);
537      } catch (InterruptedException e1) {
538        LOG.error("Unexpected interrupted exception during bulk load", e1);
539        throw (InterruptedIOException) new InterruptedIOException().initCause(e1);
540      }
541    }
542  }
543
544  @InterfaceAudience.Private
545  protected ClientServiceCallable<byte[]> buildClientServiceCallable(Connection conn,
546    TableName tableName, byte[] first, Collection<LoadQueueItem> lqis, boolean copyFile) {
547    List<Pair<byte[], String>> famPaths =
548      lqis.stream().map(lqi -> Pair.newPair(lqi.getFamily(), lqi.getFilePath().toString()))
549        .collect(Collectors.toList());
550    return new ClientServiceCallable<byte[]>(conn, tableName, first,
551      rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET) {
552      @Override
553      protected byte[] rpcCall() throws Exception {
554        SecureBulkLoadClient secureClient = null;
555        boolean success = false;
556        try {
557          if (LOG.isDebugEnabled()) {
558            LOG.debug("Going to connect to server " + getLocation() + " for row "
559              + Bytes.toStringBinary(getRow()) + " with hfile group "
560              + LoadIncrementalHFiles.this.toString(famPaths));
561          }
562          byte[] regionName = getLocation().getRegionInfo().getRegionName();
563          try (Table table = conn.getTable(getTableName())) {
564            secureClient = new SecureBulkLoadClient(getConf(), table);
565            success =
566              secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName, assignSeqIds,
567                fsDelegationToken.getUserToken(), bulkToken, copyFile, clusterIds, replicate);
568          }
569          return success ? regionName : null;
570        } finally {
571          // Best effort copying of files that might not have been imported
572          // from the staging directory back to original location
573          // in user directory
574          if (secureClient != null && !success) {
575            FileSystem targetFs = FileSystem.get(getConf());
576            FileSystem sourceFs = lqis.iterator().next().getFilePath().getFileSystem(getConf());
577            // Check to see if the source and target filesystems are the same
578            // If they are the same filesystem, we will try move the files back
579            // because previously we moved them to the staging directory.
580            if (FSUtils.isSameHdfs(getConf(), sourceFs, targetFs)) {
581              for (Pair<byte[], String> el : famPaths) {
582                Path hfileStagingPath = null;
583                Path hfileOrigPath = new Path(el.getSecond());
584                try {
585                  hfileStagingPath = new Path(new Path(bulkToken, Bytes.toString(el.getFirst())),
586                    hfileOrigPath.getName());
587                  if (targetFs.rename(hfileStagingPath, hfileOrigPath)) {
588                    LOG.debug("Moved back file " + hfileOrigPath + " from " + hfileStagingPath);
589                  } else if (targetFs.exists(hfileStagingPath)) {
590                    LOG.debug(
591                      "Unable to move back file " + hfileOrigPath + " from " + hfileStagingPath);
592                  }
593                } catch (Exception ex) {
594                  LOG.debug(
595                    "Unable to move back file " + hfileOrigPath + " from " + hfileStagingPath, ex);
596                }
597              }
598            }
599          }
600        }
601      }
602    };
603  }
604
605  private boolean
606    checkHFilesCountPerRegionPerFamily(final Multimap<ByteBuffer, LoadQueueItem> regionGroups) {
607    for (Map.Entry<ByteBuffer, Collection<LoadQueueItem>> e : regionGroups.asMap().entrySet()) {
608      Map<byte[], MutableInt> filesMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
609      for (LoadQueueItem lqi : e.getValue()) {
610        MutableInt count = filesMap.computeIfAbsent(lqi.getFamily(), k -> new MutableInt());
611        count.increment();
612        if (count.intValue() > maxFilesPerRegionPerFamily) {
613          LOG.error("Trying to load more than " + maxFilesPerRegionPerFamily + " hfiles to family "
614            + Bytes.toStringBinary(lqi.getFamily()) + " of region with start key "
615            + Bytes.toStringBinary(e.getKey()));
616          return false;
617        }
618      }
619    }
620    return true;
621  }
622
623  /**
624   * @param table        the table to load into
625   * @param pool         the ExecutorService
626   * @param queue        the queue for LoadQueueItem
627   * @param startEndKeys start and end keys
628   * @return A map that groups LQI by likely bulk load region targets and Set of missing hfiles.
629   */
630  private Pair<Multimap<ByteBuffer, LoadQueueItem>, Set<String>> groupOrSplitPhase(
631    final Table table, ExecutorService pool, Deque<LoadQueueItem> queue,
632    final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
633    // <region start key, LQI> need synchronized only within this scope of this
634    // phase because of the puts that happen in futures.
635    Multimap<ByteBuffer, LoadQueueItem> rgs = HashMultimap.create();
636    final Multimap<ByteBuffer, LoadQueueItem> regionGroups = Multimaps.synchronizedMultimap(rgs);
637    Set<String> missingHFiles = new HashSet<>();
638    Pair<Multimap<ByteBuffer, LoadQueueItem>, Set<String>> pair =
639      new Pair<>(regionGroups, missingHFiles);
640
641    // drain LQIs and figure out bulk load groups
642    Set<Future<Pair<List<LoadQueueItem>, String>>> splittingFutures = new HashSet<>();
643    while (!queue.isEmpty()) {
644      final LoadQueueItem item = queue.remove();
645
646      final Callable<Pair<List<LoadQueueItem>, String>> call =
647        new Callable<Pair<List<LoadQueueItem>, String>>() {
648          @Override
649          public Pair<List<LoadQueueItem>, String> call() throws Exception {
650            Pair<List<LoadQueueItem>, String> splits =
651              groupOrSplit(regionGroups, item, table, startEndKeys);
652            return splits;
653          }
654        };
655      splittingFutures.add(pool.submit(call));
656    }
657    // get all the results. All grouping and splitting must finish before
658    // we can attempt the atomic loads.
659    for (Future<Pair<List<LoadQueueItem>, String>> lqis : splittingFutures) {
660      try {
661        Pair<List<LoadQueueItem>, String> splits = lqis.get();
662        if (splits != null) {
663          if (splits.getFirst() != null) {
664            queue.addAll(splits.getFirst());
665          } else {
666            missingHFiles.add(splits.getSecond());
667          }
668        }
669      } catch (ExecutionException e1) {
670        Throwable t = e1.getCause();
671        if (t instanceof IOException) {
672          LOG.error("IOException during splitting", e1);
673          throw (IOException) t; // would have been thrown if not parallelized,
674        }
675        LOG.error("Unexpected execution exception during splitting", e1);
676        throw new IllegalStateException(t);
677      } catch (InterruptedException e1) {
678        LOG.error("Unexpected interrupted exception during splitting", e1);
679        throw (InterruptedIOException) new InterruptedIOException().initCause(e1);
680      }
681    }
682    return pair;
683  }
684
685  private List<LoadQueueItem> splitStoreFile(final LoadQueueItem item, final Table table,
686    byte[] startKey, byte[] splitKey) throws IOException {
687    Path hfilePath = item.getFilePath();
688    byte[] family = item.getFamily();
689    Path tmpDir = hfilePath.getParent();
690    if (!tmpDir.getName().equals(TMP_DIR)) {
691      tmpDir = new Path(tmpDir, TMP_DIR);
692    }
693
694    LOG.info("HFile at " + hfilePath + " no longer fits inside a single " + "region. Splitting...");
695
696    String uniqueName = getUniqueName();
697    ColumnFamilyDescriptor familyDesc = table.getDescriptor().getColumnFamily(family);
698
699    Path botOut = new Path(tmpDir, uniqueName + ".bottom");
700    Path topOut = new Path(tmpDir, uniqueName + ".top");
701    splitStoreFile(getConf(), hfilePath, familyDesc, splitKey, botOut, topOut);
702
703    FileSystem fs = tmpDir.getFileSystem(getConf());
704    fs.setPermission(tmpDir, FsPermission.valueOf("-rwxrwxrwx"));
705    fs.setPermission(botOut, FsPermission.valueOf("-rwxrwxrwx"));
706    fs.setPermission(topOut, FsPermission.valueOf("-rwxrwxrwx"));
707
708    // Add these back at the *front* of the queue, so there's a lower
709    // chance that the region will just split again before we get there.
710    List<LoadQueueItem> lqis = new ArrayList<>(2);
711    lqis.add(new LoadQueueItem(family, botOut));
712    lqis.add(new LoadQueueItem(family, topOut));
713
714    // If the current item is already the result of previous splits,
715    // we don't need it anymore. Clean up to save space.
716    // It is not part of the original input files.
717    try {
718      if (tmpDir.getName().equals(TMP_DIR)) {
719        fs.delete(hfilePath, false);
720      }
721    } catch (IOException e) {
722      LOG.warn("Unable to delete temporary split file " + hfilePath);
723    }
724    LOG.info("Successfully split into new HFiles " + botOut + " and " + topOut);
725    return lqis;
726  }
727
728  /**
729   * @param startEndKeys the start/end keys of regions belong to this table, the list in ascending
730   *                     order by start key
731   * @param key          the key need to find which region belong to
732   * @return region index
733   */
734  private int getRegionIndex(final Pair<byte[][], byte[][]> startEndKeys, byte[] key) {
735    int idx = Arrays.binarySearch(startEndKeys.getFirst(), key, Bytes.BYTES_COMPARATOR);
736    if (idx < 0) {
737      // not on boundary, returns -(insertion index). Calculate region it
738      // would be in.
739      idx = -(idx + 1) - 1;
740    }
741    return idx;
742  }
743
744  /**
745   * we can consider there is a region hole in following conditions. 1) if idx < 0,then first region
746   * info is lost. 2) if the endkey of a region is not equal to the startkey of the next region. 3)
747   * if the endkey of the last region is not empty.
748   */
749  private void checkRegionIndexValid(int idx, final Pair<byte[][], byte[][]> startEndKeys,
750    TableName tableName) throws IOException {
751    if (idx < 0) {
752      throw new IOException("The first region info for table " + tableName
753        + " can't be found in hbase:meta.Please use hbck tool to fix it first.");
754    } else if (
755      (idx == startEndKeys.getFirst().length - 1)
756        && !Bytes.equals(startEndKeys.getSecond()[idx], HConstants.EMPTY_BYTE_ARRAY)
757    ) {
758      throw new IOException("The last region info for table " + tableName
759        + " can't be found in hbase:meta.Please use hbck tool to fix it first.");
760    } else if (
761      idx + 1 < startEndKeys.getFirst().length
762        && !(Bytes.compareTo(startEndKeys.getSecond()[idx], startEndKeys.getFirst()[idx + 1]) == 0)
763    ) {
764      throw new IOException("The endkey of one region for table " + tableName
765        + " is not equal to the startkey of the next region in hbase:meta."
766        + "Please use hbck tool to fix it first.");
767    }
768  }
769
770  /**
771   * Attempt to assign the given load queue item into its target region group. If the hfile boundary
772   * no longer fits into a region, physically splits the hfile such that the new bottom half will
773   * fit and returns the list of LQI's corresponding to the resultant hfiles.
774   * <p>
775   * protected for testing
776   * @throws IOException if an IO failure is encountered
777   */
778  @InterfaceAudience.Private
779  protected Pair<List<LoadQueueItem>, String> groupOrSplit(
780    Multimap<ByteBuffer, LoadQueueItem> regionGroups, final LoadQueueItem item, final Table table,
781    final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
782    Path hfilePath = item.getFilePath();
783    Optional<byte[]> first, last;
784    try (HFile.Reader hfr = HFile.createReader(hfilePath.getFileSystem(getConf()), hfilePath,
785      CacheConfig.DISABLED, true, getConf())) {
786      first = hfr.getFirstRowKey();
787      last = hfr.getLastRowKey();
788    } catch (FileNotFoundException fnfe) {
789      LOG.debug("encountered", fnfe);
790      return new Pair<>(null, hfilePath.getName());
791    }
792
793    LOG.info("Trying to load hfile=" + hfilePath + " first=" + first.map(Bytes::toStringBinary)
794      + " last=" + last.map(Bytes::toStringBinary));
795    if (!first.isPresent() || !last.isPresent()) {
796      assert !first.isPresent() && !last.isPresent();
797      // TODO what if this is due to a bad HFile?
798      LOG.info("hfile " + hfilePath + " has no entries, skipping");
799      return null;
800    }
801    if (Bytes.compareTo(first.get(), last.get()) > 0) {
802      throw new IllegalArgumentException("Invalid range: " + Bytes.toStringBinary(first.get())
803        + " > " + Bytes.toStringBinary(last.get()));
804    }
805
806    int firstKeyRegionIdx = getRegionIndex(startEndKeys, first.get());
807    checkRegionIndexValid(firstKeyRegionIdx, startEndKeys, table.getName());
808    boolean lastKeyInRange =
809      Bytes.compareTo(last.get(), startEndKeys.getSecond()[firstKeyRegionIdx]) < 0
810        || Bytes.equals(startEndKeys.getSecond()[firstKeyRegionIdx], HConstants.EMPTY_BYTE_ARRAY);
811    if (!lastKeyInRange) {
812      if (failIfNeedSplitHFile) {
813        throw new IOException(
814          "The key range of hfile=" + hfilePath + " fits into no region. " + "And because "
815            + FAIL_IF_NEED_SPLIT_HFILE + " was set to true, we just skip the next steps.");
816      }
817      int lastKeyRegionIdx = getRegionIndex(startEndKeys, last.get());
818      int splitIdx = (firstKeyRegionIdx + lastKeyRegionIdx) >>> 1;
819      // make sure the splitPoint is valid in case region overlap occur, maybe the splitPoint bigger
820      // than hfile.endkey w/o this check
821      if (splitIdx != firstKeyRegionIdx) {
822        checkRegionIndexValid(splitIdx, startEndKeys, table.getName());
823      }
824      List<LoadQueueItem> lqis = splitStoreFile(item, table,
825        startEndKeys.getFirst()[firstKeyRegionIdx], startEndKeys.getSecond()[splitIdx]);
826      return new Pair<>(lqis, null);
827    }
828
829    // group regions.
830    regionGroups.put(ByteBuffer.wrap(startEndKeys.getFirst()[firstKeyRegionIdx]), item);
831    return null;
832  }
833
834  /**
835   * Attempts to do an atomic load of many hfiles into a region. If it fails, it returns a list of
836   * hfiles that need to be retried. If it is successful it will return an empty list.
837   * <p>
838   * NOTE: To maintain row atomicity guarantees, region server callable should succeed atomically
839   * and fails atomically.
840   * <p>
841   * Protected for testing.
842   * @return empty list if success, list of items to retry on recoverable failure
843   * @deprecated as of release 2.3.0. Use {@link BulkLoadHFiles} instead.
844   */
845  @Deprecated
846  @InterfaceAudience.Private
847  protected List<LoadQueueItem> tryAtomicRegionLoad(final Connection conn,
848    final TableName tableName, final byte[] first, final Collection<LoadQueueItem> lqis,
849    boolean copyFile) throws IOException {
850    ClientServiceCallable<byte[]> serviceCallable =
851      buildClientServiceCallable(conn, tableName, first, lqis, copyFile);
852    return tryAtomicRegionLoad(serviceCallable, tableName, first, lqis);
853  }
854
855  /**
856   * Attempts to do an atomic load of many hfiles into a region. If it fails, it returns a list of
857   * hfiles that need to be retried. If it is successful it will return an empty list.
858   * <p>
859   * NOTE: To maintain row atomicity guarantees, region server callable should succeed atomically
860   * and fails atomically.
861   * <p>
862   * Protected for testing.
863   * @return empty list if success, list of items to retry on recoverable failure
864   * @deprecated as of release 2.3.0. Use {@link BulkLoadHFiles} instead.
865   */
866  @Deprecated
867  @InterfaceAudience.Private
868  protected List<LoadQueueItem> tryAtomicRegionLoad(ClientServiceCallable<byte[]> serviceCallable,
869    final TableName tableName, final byte[] first, final Collection<LoadQueueItem> lqis)
870    throws IOException {
871    List<LoadQueueItem> toRetry = new ArrayList<>();
872    try {
873      Configuration conf = getConf();
874      byte[] region = RpcRetryingCallerFactory.instantiate(conf, null).<byte[]> newCaller()
875        .callWithRetries(serviceCallable, Integer.MAX_VALUE);
876      if (region == null) {
877        LOG.warn("Attempt to bulk load region containing " + Bytes.toStringBinary(first)
878          + " into table " + tableName + " with files " + lqis
879          + " failed.  This is recoverable and they will be retried.");
880        toRetry.addAll(lqis); // return lqi's to retry
881      }
882      // success
883      return toRetry;
884    } catch (IOException e) {
885      LOG.error("Encountered unrecoverable error from region server, additional details: "
886        + serviceCallable.getExceptionMessageAdditionalDetail(), e);
887      LOG.warn("Received a " + e.getClass().getSimpleName() + " from region server: "
888        + serviceCallable.getExceptionMessageAdditionalDetail(), e);
889      if (
890        getConf().getBoolean(RETRY_ON_IO_EXCEPTION, false)
891          && numRetries.get() < getConf().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
892            HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER)
893      ) {
894        LOG.warn(
895          "Will attempt to retry loading failed HFiles. Retry #" + numRetries.incrementAndGet());
896        toRetry.addAll(lqis);
897        return toRetry;
898      }
899      LOG.error(RETRY_ON_IO_EXCEPTION + " is disabled. Unable to recover");
900      throw e;
901    }
902  }
903
904  /**
905   * If the table is created for the first time, then "completebulkload" reads the files twice. More
906   * modifications necessary if we want to avoid doing it.
907   */
908  private void createTable(TableName tableName, Path hfofDir, Admin admin) throws IOException {
909    final FileSystem fs = hfofDir.getFileSystem(getConf());
910
911    // Add column families
912    // Build a set of keys
913    List<ColumnFamilyDescriptorBuilder> familyBuilders = new ArrayList<>();
914    SortedMap<byte[], Integer> map = new TreeMap<>(Bytes.BYTES_COMPARATOR);
915    visitBulkHFiles(fs, hfofDir, new BulkHFileVisitor<ColumnFamilyDescriptorBuilder>() {
916      @Override
917      public ColumnFamilyDescriptorBuilder bulkFamily(byte[] familyName) {
918        ColumnFamilyDescriptorBuilder builder =
919          ColumnFamilyDescriptorBuilder.newBuilder(familyName);
920        familyBuilders.add(builder);
921        return builder;
922      }
923
924      @Override
925      public void bulkHFile(ColumnFamilyDescriptorBuilder builder, FileStatus hfileStatus)
926        throws IOException {
927        Path hfile = hfileStatus.getPath();
928        try (HFile.Reader reader =
929          HFile.createReader(fs, hfile, CacheConfig.DISABLED, true, getConf())) {
930          if (builder.getCompressionType() != reader.getFileContext().getCompression()) {
931            builder.setCompressionType(reader.getFileContext().getCompression());
932            LOG.info("Setting compression " + reader.getFileContext().getCompression().name()
933              + " for family " + builder.getNameAsString());
934          }
935          byte[] first = reader.getFirstRowKey().get();
936          byte[] last = reader.getLastRowKey().get();
937
938          LOG.info("Trying to figure out region boundaries hfile=" + hfile + " first="
939            + Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last));
940
941          // To eventually infer start key-end key boundaries
942          Integer value = map.containsKey(first) ? map.get(first) : 0;
943          map.put(first, value + 1);
944
945          value = map.containsKey(last) ? map.get(last) : 0;
946          map.put(last, value - 1);
947        }
948      }
949    });
950
951    byte[][] keys = inferBoundaries(map);
952    TableDescriptorBuilder tdBuilder = TableDescriptorBuilder.newBuilder(tableName);
953    familyBuilders.stream().map(ColumnFamilyDescriptorBuilder::build)
954      .forEachOrdered(tdBuilder::setColumnFamily);
955    admin.createTable(tdBuilder.build(), keys);
956
957    LOG.info("Table " + tableName + " is available!!");
958  }
959
960  private void cleanup(Admin admin, Deque<LoadQueueItem> queue, ExecutorService pool,
961    SecureBulkLoadClient secureClient) throws IOException {
962    fsDelegationToken.releaseDelegationToken();
963    if (bulkToken != null && secureClient != null) {
964      secureClient.cleanupBulkLoad(admin.getConnection(), bulkToken);
965    }
966    if (pool != null) {
967      pool.shutdown();
968    }
969    if (!queue.isEmpty()) {
970      StringBuilder err = new StringBuilder();
971      err.append("-------------------------------------------------\n");
972      err.append("Bulk load aborted with some files not yet loaded:\n");
973      err.append("-------------------------------------------------\n");
974      for (LoadQueueItem q : queue) {
975        err.append("  ").append(q.getFilePath()).append('\n');
976      }
977      LOG.error(err.toString());
978    }
979  }
980
981  // unique file name for the table
982  private String getUniqueName() {
983    return UUID.randomUUID().toString().replaceAll("-", "");
984  }
985
986  /**
987   * Checks whether there is any invalid family name in HFiles to be bulk loaded.
988   */
989  private void validateFamiliesInHFiles(Table table, Deque<LoadQueueItem> queue, boolean silence)
990    throws IOException {
991    Set<String> familyNames = Arrays.asList(table.getDescriptor().getColumnFamilies()).stream()
992      .map(f -> f.getNameAsString()).collect(Collectors.toSet());
993    List<String> unmatchedFamilies = queue.stream().map(item -> Bytes.toString(item.getFamily()))
994      .filter(fn -> !familyNames.contains(fn)).distinct().collect(Collectors.toList());
995    if (unmatchedFamilies.size() > 0) {
996      String msg =
997        "Unmatched family names found: unmatched family names in HFiles to be bulkloaded: "
998          + unmatchedFamilies + "; valid family names of table " + table.getName() + " are: "
999          + familyNames;
1000      LOG.error(msg);
1001      if (!silence) {
1002        throw new IOException(msg);
1003      }
1004    }
1005  }
1006
1007  /**
1008   * Populate the Queue with given HFiles
1009   */
1010  private void populateLoadQueue(Deque<LoadQueueItem> ret, Map<byte[], List<Path>> map) {
1011    map.forEach((k, v) -> v.stream().map(p -> new LoadQueueItem(k, p)).forEachOrdered(ret::add));
1012  }
1013
1014  /**
1015   * Walk the given directory for all HFiles, and return a Queue containing all such files.
1016   */
1017  private void discoverLoadQueue(final Deque<LoadQueueItem> ret, final Path hfofDir,
1018    final boolean validateHFile) throws IOException {
1019    visitBulkHFiles(hfofDir.getFileSystem(getConf()), hfofDir, new BulkHFileVisitor<byte[]>() {
1020      @Override
1021      public byte[] bulkFamily(final byte[] familyName) {
1022        return familyName;
1023      }
1024
1025      @Override
1026      public void bulkHFile(final byte[] family, final FileStatus hfile) throws IOException {
1027        long length = hfile.getLen();
1028        if (
1029          length
1030              > getConf().getLong(HConstants.HREGION_MAX_FILESIZE, HConstants.DEFAULT_MAX_FILE_SIZE)
1031        ) {
1032          LOG.warn("Trying to bulk load hfile " + hfile.getPath() + " with size: " + length
1033            + " bytes can be problematic as it may lead to oversplitting.");
1034        }
1035        ret.add(new LoadQueueItem(family, hfile.getPath()));
1036      }
1037    }, validateHFile);
1038  }
1039
1040  private interface BulkHFileVisitor<TFamily> {
1041
1042    TFamily bulkFamily(byte[] familyName) throws IOException;
1043
1044    void bulkHFile(TFamily family, FileStatus hfileStatus) throws IOException;
1045  }
1046
1047  /**
1048   * Iterate over the bulkDir hfiles. Skip reference, HFileLink, files starting with "_" and
1049   * non-valid hfiles.
1050   */
1051  private static <TFamily> void visitBulkHFiles(final FileSystem fs, final Path bulkDir,
1052    final BulkHFileVisitor<TFamily> visitor) throws IOException {
1053    visitBulkHFiles(fs, bulkDir, visitor, true);
1054  }
1055
1056  /**
1057   * Iterate over the bulkDir hfiles. Skip reference, HFileLink, files starting with "_". Check and
1058   * skip non-valid hfiles by default, or skip this validation by setting
1059   * 'hbase.loadincremental.validate.hfile' to false.
1060   */
1061  private static <TFamily> void visitBulkHFiles(FileSystem fs, Path bulkDir,
1062    BulkHFileVisitor<TFamily> visitor, boolean validateHFile) throws IOException {
1063    FileStatus[] familyDirStatuses = fs.listStatus(bulkDir);
1064    for (FileStatus familyStat : familyDirStatuses) {
1065      if (!familyStat.isDirectory()) {
1066        LOG.warn("Skipping non-directory " + familyStat.getPath());
1067        continue;
1068      }
1069      Path familyDir = familyStat.getPath();
1070      byte[] familyName = Bytes.toBytes(familyDir.getName());
1071      // Skip invalid family
1072      try {
1073        ColumnFamilyDescriptorBuilder.isLegalColumnFamilyName(familyName);
1074      } catch (IllegalArgumentException e) {
1075        LOG.warn("Skipping invalid " + familyStat.getPath());
1076        continue;
1077      }
1078      TFamily family = visitor.bulkFamily(familyName);
1079
1080      FileStatus[] hfileStatuses = fs.listStatus(familyDir);
1081      for (FileStatus hfileStatus : hfileStatuses) {
1082        if (!fs.isFile(hfileStatus.getPath())) {
1083          LOG.warn("Skipping non-file " + hfileStatus);
1084          continue;
1085        }
1086
1087        Path hfile = hfileStatus.getPath();
1088        // Skip "_", reference, HFileLink
1089        String fileName = hfile.getName();
1090        if (fileName.startsWith("_")) {
1091          continue;
1092        }
1093        if (StoreFileInfo.isReference(fileName)) {
1094          LOG.warn("Skipping reference " + fileName);
1095          continue;
1096        }
1097        if (HFileLink.isHFileLink(fileName)) {
1098          LOG.warn("Skipping HFileLink " + fileName);
1099          continue;
1100        }
1101
1102        // Validate HFile Format if needed
1103        if (validateHFile) {
1104          try {
1105            if (!HFile.isHFileFormat(fs, hfile)) {
1106              LOG.warn("the file " + hfile + " doesn't seems to be an hfile. skipping");
1107              continue;
1108            }
1109          } catch (FileNotFoundException e) {
1110            LOG.warn("the file " + hfile + " was removed");
1111            continue;
1112          }
1113        }
1114
1115        visitor.bulkHFile(family, hfileStatus);
1116      }
1117    }
1118  }
1119
1120  // Initialize a thread pool
1121  private ExecutorService createExecutorService() {
1122    ThreadPoolExecutor pool = new ThreadPoolExecutor(nrThreads, nrThreads, 60, TimeUnit.SECONDS,
1123      new LinkedBlockingQueue<>(),
1124      new ThreadFactoryBuilder().setNameFormat("LoadIncrementalHFiles-%1$d").build());
1125    pool.allowCoreThreadTimeOut(true);
1126    return pool;
1127  }
1128
1129  private final String toString(List<Pair<byte[], String>> list) {
1130    StringBuilder sb = new StringBuilder();
1131    sb.append('[');
1132    list.forEach(p -> {
1133      sb.append('{').append(Bytes.toStringBinary(p.getFirst())).append(',').append(p.getSecond())
1134        .append('}');
1135    });
1136    sb.append(']');
1137    return sb.toString();
1138  }
1139
1140  private boolean isSecureBulkLoadEndpointAvailable() {
1141    String classes = getConf().get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
1142    return classes.contains("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
1143  }
1144
1145  /**
1146   * Split a storefile into a top and bottom half, maintaining the metadata, recreating bloom
1147   * filters, etc.
1148   */
1149  @InterfaceAudience.Private
1150  static void splitStoreFile(Configuration conf, Path inFile, ColumnFamilyDescriptor familyDesc,
1151    byte[] splitKey, Path bottomOut, Path topOut) throws IOException {
1152    // Open reader with no block cache, and not in-memory
1153    Reference topReference = Reference.createTopReference(splitKey);
1154    Reference bottomReference = Reference.createBottomReference(splitKey);
1155
1156    copyHFileHalf(conf, inFile, topOut, topReference, familyDesc);
1157    copyHFileHalf(conf, inFile, bottomOut, bottomReference, familyDesc);
1158  }
1159
1160  /**
1161   * Copy half of an HFile into a new HFile.
1162   */
1163  private static void copyHFileHalf(Configuration conf, Path inFile, Path outFile,
1164    Reference reference, ColumnFamilyDescriptor familyDescriptor) throws IOException {
1165    FileSystem fs = inFile.getFileSystem(conf);
1166    CacheConfig cacheConf = CacheConfig.DISABLED;
1167    HalfStoreFileReader halfReader = null;
1168    StoreFileWriter halfWriter = null;
1169    try {
1170      ReaderContext context = new ReaderContextBuilder().withFileSystemAndPath(fs, inFile).build();
1171      HFileInfo hfile = new HFileInfo(context, conf);
1172      halfReader =
1173        new HalfStoreFileReader(context, hfile, cacheConf, reference, new AtomicInteger(0), conf);
1174      hfile.initMetaAndIndex(halfReader.getHFileReader());
1175      Map<byte[], byte[]> fileInfo = halfReader.loadFileInfo();
1176
1177      int blocksize = familyDescriptor.getBlocksize();
1178      Algorithm compression = familyDescriptor.getCompressionType();
1179      BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
1180      HFileContext hFileContext = new HFileContextBuilder().withCompression(compression)
1181        .withChecksumType(StoreUtils.getChecksumType(conf))
1182        .withBytesPerCheckSum(StoreUtils.getBytesPerChecksum(conf)).withBlockSize(blocksize)
1183        .withDataBlockEncoding(familyDescriptor.getDataBlockEncoding()).withIncludesTags(true)
1184        .build();
1185      halfWriter = new StoreFileWriter.Builder(conf, cacheConf, fs).withFilePath(outFile)
1186        .withBloomType(bloomFilterType).withFileContext(hFileContext).build();
1187      HFileScanner scanner = halfReader.getScanner(false, false, false);
1188      scanner.seekTo();
1189      do {
1190        halfWriter.append(scanner.getCell());
1191      } while (scanner.next());
1192
1193      for (Map.Entry<byte[], byte[]> entry : fileInfo.entrySet()) {
1194        if (shouldCopyHFileMetaKey(entry.getKey())) {
1195          halfWriter.appendFileInfo(entry.getKey(), entry.getValue());
1196        }
1197      }
1198    } finally {
1199      if (halfReader != null) {
1200        try {
1201          halfReader.close(cacheConf.shouldEvictOnClose());
1202        } catch (IOException e) {
1203          LOG.warn("failed to close hfile reader for " + inFile, e);
1204        }
1205      }
1206      if (halfWriter != null) {
1207        halfWriter.close();
1208      }
1209
1210    }
1211  }
1212
1213  private static boolean shouldCopyHFileMetaKey(byte[] key) {
1214    // skip encoding to keep hfile meta consistent with data block info, see HBASE-15085
1215    if (Bytes.equals(key, HFileDataBlockEncoder.DATA_BLOCK_ENCODING)) {
1216      return false;
1217    }
1218
1219    return !HFileInfo.isReservedFileInfoKey(key);
1220  }
1221
1222  private boolean isCreateTable() {
1223    return "yes".equalsIgnoreCase(getConf().get(CREATE_TABLE_CONF_KEY, "yes"));
1224  }
1225
1226  private boolean isSilence() {
1227    return "yes".equalsIgnoreCase(getConf().get(IGNORE_UNMATCHED_CF_CONF_KEY, ""));
1228  }
1229
1230  private boolean isAlwaysCopyFiles() {
1231    return getConf().getBoolean(ALWAYS_COPY_FILES, false);
1232  }
1233
1234  protected final Map<LoadQueueItem, ByteBuffer> run(Path hfofDir, TableName tableName)
1235    throws IOException {
1236    try (Connection connection = ConnectionFactory.createConnection(getConf());
1237      Admin admin = connection.getAdmin()) {
1238      if (!admin.tableExists(tableName)) {
1239        if (isCreateTable()) {
1240          createTable(tableName, hfofDir, admin);
1241        } else {
1242          String errorMsg = format("Table '%s' does not exist.", tableName);
1243          LOG.error(errorMsg);
1244          throw new TableNotFoundException(errorMsg);
1245        }
1246      }
1247      try (Table table = connection.getTable(tableName);
1248        RegionLocator locator = connection.getRegionLocator(tableName)) {
1249        return doBulkLoad(hfofDir, admin, table, locator, isSilence(), isAlwaysCopyFiles());
1250      }
1251    }
1252  }
1253
1254  /**
1255   * Perform bulk load on the given table.
1256   * @param hfofDir   the directory that was provided as the output path of a job using
1257   *                  HFileOutputFormat
1258   * @param tableName the table to load into
1259   */
1260  public Map<LoadQueueItem, ByteBuffer> run(String hfofDir, TableName tableName)
1261    throws IOException {
1262    return run(new Path(hfofDir), tableName);
1263  }
1264
1265  /**
1266   * Perform bulk load on the given table.
1267   * @param family2Files map of family to List of hfiles
1268   * @param tableName    the table to load into
1269   */
1270  public Map<LoadQueueItem, ByteBuffer> run(Map<byte[], List<Path>> family2Files,
1271    TableName tableName) throws IOException {
1272    try (Connection connection = ConnectionFactory.createConnection(getConf());
1273      Admin admin = connection.getAdmin()) {
1274      if (!admin.tableExists(tableName)) {
1275        String errorMsg = format("Table '%s' does not exist.", tableName);
1276        LOG.error(errorMsg);
1277        throw new TableNotFoundException(errorMsg);
1278      }
1279      try (Table table = connection.getTable(tableName);
1280        RegionLocator locator = connection.getRegionLocator(tableName)) {
1281        return doBulkLoad(family2Files, admin, table, locator, isSilence(), isAlwaysCopyFiles());
1282      }
1283    }
1284  }
1285
1286  @Override
1287  public int run(String[] args) throws Exception {
1288    if (args.length != 2 && args.length != 3) {
1289      usage();
1290      return -1;
1291    }
1292    // Re-initialize to apply -D options from the command line parameters
1293    initialize();
1294    String dirPath = args[0];
1295    TableName tableName = TableName.valueOf(args[1]);
1296    if (args.length == 2) {
1297      return !run(dirPath, tableName).isEmpty() ? 0 : -1;
1298    } else {
1299      Map<byte[], List<Path>> family2Files = Maps.newHashMap();
1300      FileSystem fs = FileSystem.get(getConf());
1301      for (FileStatus regionDir : fs.listStatus(new Path(dirPath))) {
1302        FSVisitor.visitRegionStoreFiles(fs, regionDir.getPath(), (region, family, hfileName) -> {
1303          Path path = new Path(regionDir.getPath(), new Path(family, hfileName));
1304          byte[] familyName = Bytes.toBytes(family);
1305          if (family2Files.containsKey(familyName)) {
1306            family2Files.get(familyName).add(path);
1307          } else {
1308            family2Files.put(familyName, Lists.newArrayList(path));
1309          }
1310        });
1311      }
1312      return !run(family2Files, tableName).isEmpty() ? 0 : -1;
1313    }
1314
1315  }
1316
1317  public static void main(String[] args) throws Exception {
1318    Configuration conf = HBaseConfiguration.create();
1319    int ret = ToolRunner.run(conf, new LoadIncrementalHFiles(conf), args);
1320    System.exit(ret);
1321  }
1322
1323  /**
1324   * Called from replication sink, where it manages bulkToken(staging directory) by itself. This is
1325   * used only when SecureBulkLoadEndpoint is configured in hbase.coprocessor.region.classes
1326   * property. This directory is used as a temporary directory where all files are initially
1327   * copied/moved from user given directory, set all the required file permissions and then from
1328   * their it is finally loaded into a table. This should be set only when, one would like to manage
1329   * the staging directory by itself. Otherwise this tool will handle this by itself.
1330   * @param stagingDir staging directory path
1331   */
1332  public void setBulkToken(String stagingDir) {
1333    this.bulkToken = stagingDir;
1334  }
1335
1336  public void setClusterIds(List<String> clusterIds) {
1337    this.clusterIds = clusterIds;
1338  }
1339
1340  /**
1341   * Disables replication for these bulkloaded files.
1342   */
1343  public void disableReplication() {
1344    this.replicate = false;
1345  }
1346
1347  /**
1348   * Infers region boundaries for a new table.
1349   * <p>
1350   * Parameter: <br>
1351   * bdryMap is a map between keys to an integer belonging to {+1, -1}
1352   * <ul>
1353   * <li>If a key is a start key of a file, then it maps to +1</li>
1354   * <li>If a key is an end key of a file, then it maps to -1</li>
1355   * </ul>
1356   * <p>
1357   * Algo:<br>
1358   * <ol>
1359   * <li>Poll on the keys in order:
1360   * <ol type="a">
1361   * <li>Keep adding the mapped values to these keys (runningSum)</li>
1362   * <li>Each time runningSum reaches 0, add the start Key from when the runningSum had started to a
1363   * boundary list.</li>
1364   * </ol>
1365   * </li>
1366   * <li>Return the boundary list.</li>
1367   * </ol>
1368   */
1369  public static byte[][] inferBoundaries(SortedMap<byte[], Integer> bdryMap) {
1370    List<byte[]> keysArray = new ArrayList<>();
1371    int runningValue = 0;
1372    byte[] currStartKey = null;
1373    boolean firstBoundary = true;
1374
1375    for (Map.Entry<byte[], Integer> item : bdryMap.entrySet()) {
1376      if (runningValue == 0) {
1377        currStartKey = item.getKey();
1378      }
1379      runningValue += item.getValue();
1380      if (runningValue == 0) {
1381        if (!firstBoundary) {
1382          keysArray.add(currStartKey);
1383        }
1384        firstBoundary = false;
1385      }
1386    }
1387
1388    return keysArray.toArray(new byte[0][]);
1389  }
1390}