You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by wc...@apache.org on 2020/05/04 14:20:11 UTC

[hbase] 01/01: Revert "HBASE-24221 Support bulkLoadHFile by family, fix UT (#1633)"

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

wchevreuil pushed a commit to branch revert-1633-HBASE-24221
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 694d1fd56844a09bde06c564cb50ac468c7a09ac
Author: Wellington Ramos Chevreuil <we...@gmail.com>
AuthorDate: Mon May 4 15:20:01 2020 +0100

    Revert "HBASE-24221 Support bulkLoadHFile by family, fix UT (#1633)"
    
    This reverts commit 5d41588df6fd4165138a40c55257e3eb29b1acc9.
---
 .../hadoop/hbase/tool/BulkLoadHFilesTool.java      |  8 +---
 .../hadoop/hbase/tool/TestBulkLoadHFiles.java      | 43 +--------------------
 .../hbase/tool/TestBulkLoadHFilesByFamily.java     | 45 ++++++++++++++++++++++
 3 files changed, 47 insertions(+), 49 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 e8b701b..b42fbbe 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
@@ -374,15 +374,9 @@ public class BulkLoadHFilesTool extends Configured implements BulkLoadHFiles, To
    * hfiles that need to be retried. If it is successful it will return an empty list. NOTE: To
    * maintain row atomicity guarantees, region server side should succeed atomically and fails
    * atomically.
-   * @param conn Connection to use
-   * @param tableName Table to which these hfiles should be loaded to
-   * @param copyFiles whether replicate to peer cluster while bulkloading
-   * @param first the start key of region
-   * @param lqis hfiles should be loaded
    * @return empty list if success, list of items to retry on recoverable failure
    */
-  @VisibleForTesting
-  protected CompletableFuture<Collection<LoadQueueItem>> tryAtomicRegionLoad(
+  private CompletableFuture<Collection<LoadQueueItem>> tryAtomicRegionLoad(
       final AsyncClusterConnection conn, final TableName tableName, boolean copyFiles,
       final byte[] first, Collection<LoadQueueItem> lqis) {
     List<Pair<byte[], String>> familyPaths =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFiles.java
index d90d75f..6d0e914 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFiles.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFiles.java
@@ -26,13 +26,10 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.TreeMap;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -44,7 +41,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.AsyncClusterConnection;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Table;
@@ -106,6 +102,7 @@ public class TestBulkLoadHFiles {
     // change default behavior so that tag values are returned with normal rpcs
     util.getConfiguration().set(HConstants.RPC_CODEC_CONF_KEY,
       KeyValueCodecWithTags.class.getCanonicalName());
+    util.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, false);
     util.startMiniCluster();
 
     setupNamespace();
@@ -744,42 +741,4 @@ public class TestBulkLoadHFiles {
       assertEquals(1000, countRows(table));
     }
   }
-
-  @Test
-  public void testBulkLoadByFamily() throws Exception {
-    Path dir = util.getDataTestDirOnTestFS("testBulkLoadByFamily");
-    FileSystem fs = util.getTestFileSystem();
-    dir = dir.makeQualified(fs.getUri(), fs.getWorkingDirectory());
-    String tableName = tn.getMethodName();
-    String[] families = { "cf1", "cf2", "cf3" };
-    for (int i = 0; i < families.length; i++) {
-      byte[] from = Bytes.toBytes(i + "begin");
-      byte[] to = Bytes.toBytes(i + "end");
-      Path familyDir = new Path(dir, families[i]);
-      HFileTestUtil.createHFile(util.getConfiguration(), fs, new Path(familyDir, "hfile"),
-        Bytes.toBytes(families[i]), QUALIFIER, from, to, 1000);
-    }
-    Table table = util.createTable(TableName.valueOf(tableName), families);
-    final AtomicInteger attmptedCalls = new AtomicInteger();
-    util.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, true);
-    BulkLoadHFiles loader = new BulkLoadHFilesTool(util.getConfiguration()) {
-      @Override
-      protected CompletableFuture<Collection<LoadQueueItem>> tryAtomicRegionLoad(
-          final AsyncClusterConnection conn, final TableName tableName, boolean copyFiles,
-          final byte[] first, Collection<LoadQueueItem> lqis) {
-        attmptedCalls.incrementAndGet();
-        return super.tryAtomicRegionLoad(conn, tableName, copyFiles, first, lqis);
-      }
-    };
-    try {
-      loader.bulkLoad(table.getName(), dir);
-      assertEquals(families.length, attmptedCalls.get());
-      assertEquals(1000 * families.length, HBaseTestingUtility.countRows(table));
-    } finally {
-      if (null != table) {
-        table.close();
-      }
-      util.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, false);
-    }
-  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesByFamily.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesByFamily.java
new file mode 100644
index 0000000..42a81a9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestBulkLoadHFilesByFamily.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.tool;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+
+public class TestBulkLoadHFilesByFamily extends TestBulkLoadHFiles {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestBulkLoadHFilesByFamily.class);
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
+    util.getConfiguration().setInt(BulkLoadHFiles.MAX_FILES_PER_REGION_PER_FAMILY,
+      MAX_FILES_PER_REGION_PER_FAMILY);
+    // change default behavior so that tag values are returned with normal rpcs
+    util.getConfiguration().set(HConstants.RPC_CODEC_CONF_KEY,
+      KeyValueCodecWithTags.class.getCanonicalName());
+    util.getConfiguration().setBoolean(BulkLoadHFilesTool.BULK_LOAD_HFILES_BY_FAMILY, true);
+    util.startMiniCluster();
+    setupNamespace();
+  }
+}