You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2019/09/09 00:28:12 UTC

[hbase] branch master updated: HBASE-22989 : null check for item2RegionMap while removing LoadQueueItem (#596)

This is an automated email from the ASF dual-hosted git repository.

zhangduo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new cb976b0  HBASE-22989 : null check for item2RegionMap while removing LoadQueueItem (#596)
cb976b0 is described below

commit cb976b030688cd266c1eba852b9d9551c448657a
Author: Viraj Jasani <vi...@gmail.com>
AuthorDate: Mon Sep 9 05:58:06 2019 +0530

    HBASE-22989 : null check for item2RegionMap while removing LoadQueueItem (#596)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../hadoop/hbase/tool/BulkLoadHFilesTool.java      |  6 ++-
 .../tool/TestBulkLoadHFilesSplitRecovery.java      | 49 ++++++++++++++++++++++
 2 files changed, 53 insertions(+), 2 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java
index 7c0c7af..6d5b11b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java
@@ -413,8 +413,10 @@ public class BulkLoadHFilesTool extends Configured implements BulkLoadHFiles, To
       try {
         Collection<LoadQueueItem> toRetry = future.get();
 
-        for (LoadQueueItem lqi : toRetry) {
-          item2RegionMap.remove(lqi);
+        if (item2RegionMap != null) {
+          for (LoadQueueItem lqi : toRetry) {
+            item2RegionMap.remove(lqi);
+          }
         }
         // LQIs that are requeued to be regrouped.
         queue.addAll(toRetry);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesSplitRecovery.java
index 2aef16e..f578fec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesSplitRecovery.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesSplitRecovery.java
@@ -465,6 +465,55 @@ public class TestBulkLoadHFilesSplitRecovery {
   }
 
   /**
+   * We are testing a split after initial validation but before the atomic bulk load call.
+   * We cannot use presplitting to test this path, so we actually inject a
+   * split just before the atomic region load. However, we will pass null item2RegionMap
+   * and that should not affect the bulk load behavior.
+   */
+  @Test
+  public void testSplitWhileBulkLoadPhaseWithoutItemMap() throws Exception {
+    final TableName table = TableName.valueOf(name.getMethodName());
+    setupTable(util.getConnection(), table, 10);
+    populateTable(util.getConnection(), table, 1);
+    assertExpectedTable(table, ROWCOUNT, 1);
+
+    // Now let's cause trouble. This will occur after checks and cause bulk
+    // files to fail when attempt to atomically import. This is recoverable.
+    final AtomicInteger attemptedCalls = new AtomicInteger();
+    BulkLoadHFilesTool loader = new BulkLoadHFilesTool(util.getConfiguration()) {
+
+      @Override
+      protected void bulkLoadPhase(final AsyncClusterConnection conn, final TableName tableName,
+        final Deque<LoadQueueItem> queue, final Multimap<ByteBuffer, LoadQueueItem> regionGroups,
+        final boolean copyFiles,
+        final Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
+
+        int i = attemptedCalls.incrementAndGet();
+        if (i == 1) {
+          // On first attempt force a split.
+          forceSplit(table);
+        }
+
+        // Passing item2RegionMap null
+        // In the absence of LoadQueueItem, bulk load should work as expected
+        super.bulkLoadPhase(conn, tableName, queue, regionGroups, copyFiles, null);
+      }
+
+    };
+
+    // create HFiles for different column families
+    Path dir = buildBulkFiles(table, 2);
+    loader.bulkLoad(table, dir);
+
+    // check that data was loaded
+    // The three expected attempts are 1) failure because need to split, 2)
+    // load of split top 3) load of split bottom
+    assertEquals(3, attemptedCalls.get());
+    assertExpectedTable(table, ROWCOUNT, 2);
+  }
+
+
+  /**
    * Checks that all columns have the expected value and that there is the expected number of rows.
    */
   void assertExpectedTable(final Connection connection, TableName table, int count, int value)