You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2017/07/10 16:05:02 UTC

hbase git commit: HBASE-17705 Procedure execution must fail fast if procedure is not registered (Vladimir Rodionov)

Repository: hbase
Updated Branches:
  refs/heads/master 351703455 -> 7d007eac9


HBASE-17705 Procedure execution must fail fast if procedure is not registered (Vladimir Rodionov)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/7d007eac
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/7d007eac
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/7d007eac

Branch: refs/heads/master
Commit: 7d007eac98daef8d41ac1d8adf397967fa2919fd
Parents: 3517034
Author: tedyu <yu...@gmail.com>
Authored: Mon Jul 10 09:04:56 2017 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Jul 10 09:04:56 2017 -0700

----------------------------------------------------------------------
 .../hbase/client/RpcRetryingCallerImpl.java     |  6 ++-
 .../hadoop/hbase/master/MasterRpcServices.java  |  4 +-
 .../TestFastFailOnProcedureNotRegistered.java   | 42 ++++++++++++++++++++
 3 files changed, 49 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7d007eac/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
index c59b020..22b79cf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
@@ -109,7 +109,11 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
       } catch (Throwable t) {
         Throwable e = t.getCause();
         ExceptionUtil.rethrowIfInterrupt(t);
-
+        Throwable cause = t.getCause();
+        if (cause instanceof DoNotRetryIOException) {
+          // Fail fast
+          throw (DoNotRetryIOException) cause;
+        }
         // translateException throws exception when should not retry: i.e. when request is bad.
         interceptor.handleFailure(context, t);
         t = translateException(t);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7d007eac/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index c6397f3..64b7757 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -702,8 +702,8 @@ public class MasterRpcServices extends RSRpcServices
       MasterProcedureManager mpm = master.getMasterProcedureManagerHost().getProcedureManager(
         desc.getSignature());
       if (mpm == null) {
-        throw new ServiceException("The procedure is not registered: "
-          + desc.getSignature());
+        throw new ServiceException(new DoNotRetryIOException("The procedure is not registered: "
+          + desc.getSignature()));
       }
 
       LOG.info(master.getClientIdAuditPrefix() + " procedure request for: "

http://git-wip-us.apache.org/repos/asf/hbase/blob/7d007eac/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestFastFailOnProcedureNotRegistered.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestFastFailOnProcedureNotRegistered.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestFastFailOnProcedureNotRegistered.java
new file mode 100644
index 0000000..363eba8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestFastFailOnProcedureNotRegistered.java
@@ -0,0 +1,42 @@
+/**
+ * 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.procedure;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestFastFailOnProcedureNotRegistered extends TestTableDDLProcedureBase {
+
+  @Test(expected=DoNotRetryIOException.class, timeout = 3000)
+  public void testFastFailOnProcedureNotRegistered() throws IOException {
+    Admin admin = UTIL.getAdmin();
+    Map<String, String> props = new HashMap<String, String>();
+    admin.execProcedure("fake1","fake2", props);
+  }
+
+}