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/30 04:00:38 UTC

[hbase] branch branch-2.2 updated: HBASE-23079 RegionRemoteProcedureBase should override setTimeoutFailure (#672)

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

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


The following commit(s) were added to refs/heads/branch-2.2 by this push:
     new b29844f  HBASE-23079 RegionRemoteProcedureBase should override setTimeoutFailure (#672)
b29844f is described below

commit b29844f5bd8dd14303e0f43e6e949bf0272869ec
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Mon Sep 30 11:48:50 2019 +0800

    HBASE-23079 RegionRemoteProcedureBase should override setTimeoutFailure (#672)
    
    Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
 .../assignment/RegionRemoteProcedureBase.java      |   7 ++
 .../assignment/TestOpenRegionProcedureBackoff.java | 125 +++++++++++++++++++++
 2 files changed, 132 insertions(+)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
index a3c3382..ee37746 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
@@ -325,6 +325,13 @@ public abstract class RegionRemoteProcedureBase extends Procedure<MasterProcedur
   }
 
   @Override
+  protected synchronized boolean setTimeoutFailure(MasterProcedureEnv env) {
+    setState(ProcedureProtos.ProcedureState.RUNNABLE);
+    env.getProcedureScheduler().addFront(this);
+    return false; // 'false' means that this procedure handled the timeout
+  }
+
+  @Override
   public boolean storeInDispatchedQueue() {
     return false;
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java
new file mode 100644
index 0000000..69ad093
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java
@@ -0,0 +1,125 @@
+/**
+ * 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.master.assignment;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ProcedureTestUtil;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncAdmin;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.zookeeper.KeeperException;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Testcase for HBASE-23079.
+ */
+@Category({ MasterTests.class, MediumTests.class })
+public class TestOpenRegionProcedureBackoff {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestOpenRegionProcedureBackoff.class);
+
+  private static volatile boolean FAIL = false;
+
+  private static final class AssignmentManagerForTest extends AssignmentManager {
+
+    public AssignmentManagerForTest(MasterServices master) {
+      super(master);
+    }
+
+    @Override
+    void persistToMeta(RegionStateNode regionNode) throws IOException {
+      if (FAIL) {
+        throw new IOException("Inject Error!");
+      }
+      super.persistToMeta(regionNode);
+    }
+  }
+
+  public static final class HMasterForTest extends HMaster {
+
+    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+      super(conf);
+    }
+
+    @Override
+    protected AssignmentManager createAssignmentManager(MasterServices master) {
+      return new AssignmentManagerForTest(master);
+    }
+  }
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static TableName NAME = TableName.valueOf("Open");
+
+  private static byte[] CF = Bytes.toBytes("cf");
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration conf = UTIL.getConfiguration();
+    conf.setClass(HConstants.MASTER_IMPL, HMasterForTest.class, HMaster.class);
+    UTIL.startMiniCluster(1);
+    UTIL.waitTableAvailable(TableName.META_TABLE_NAME);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  private void assertBackoffIncrease() throws IOException, InterruptedException {
+    ProcedureTestUtil.waitUntilProcedureWaitingTimeout(UTIL, OpenRegionProcedure.class, 30000);
+    ProcedureTestUtil.waitUntilProcedureTimeoutIncrease(UTIL, OpenRegionProcedure.class, 2);
+  }
+
+  @Test
+  public void testBackoff() throws IOException, InterruptedException, ExecutionException {
+    FAIL = true;
+    try (AsyncConnection conn =
+      ConnectionFactory.createAsyncConnection(UTIL.getConfiguration()).get()) {
+      AsyncAdmin admin = conn.getAdminBuilder().setRpcTimeout(5, TimeUnit.MINUTES)
+        .setOperationTimeout(10, TimeUnit.MINUTES).build();
+      CompletableFuture<?> future = admin.createTable(TableDescriptorBuilder.newBuilder(NAME)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(CF)).build());
+      assertBackoffIncrease();
+      FAIL = false;
+      future.get();
+      UTIL.waitTableAvailable(NAME);
+    }
+  }
+}