You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by vj...@apache.org on 2023/09/29 05:18:11 UTC

[hbase] branch branch-3 updated: HBASE-28050 RSProcedureDispatcher to fail-fast for krb auth failures (#5391)

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

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


The following commit(s) were added to refs/heads/branch-3 by this push:
     new 50b2b98082d HBASE-28050 RSProcedureDispatcher to fail-fast for krb auth failures (#5391)
50b2b98082d is described below

commit 50b2b98082de488e7c0621c08b81603d2a2934be
Author: Viraj Jasani <vj...@apache.org>
AuthorDate: Thu Sep 28 21:15:50 2023 -0800

    HBASE-28050 RSProcedureDispatcher to fail-fast for krb auth failures (#5391)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Andrew Purtell <ap...@apache.org>
    Signed-off-by: Aman Poonia <am...@gmail.com>
    Signed-off-by: David Manning <da...@salesforce.com>
---
 .../hadoop/hbase/ipc/NettyRpcConnection.java       |  2 +-
 .../hadoop/hbase/ipc/RpcConnectionConstants.java   | 34 +++++++++++++
 .../master/procedure/RSProcedureDispatcher.java    | 59 ++++++++++++++++++----
 3 files changed, 85 insertions(+), 10 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
index 3f9a58d5126..408ea347e7a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
@@ -347,7 +347,7 @@ class NettyRpcConnection extends RpcConnection {
   private void sendRequest0(Call call, HBaseRpcController hrc) throws IOException {
     assert eventLoop.inEventLoop();
     if (reloginInProgress) {
-      throw new IOException("Can not send request because relogin is in progress.");
+      throw new IOException(RpcConnectionConstants.RELOGIN_IS_IN_PROGRESS);
     }
     hrc.notifyOnCancel(new RpcCallback<Object>() {
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnectionConstants.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnectionConstants.java
new file mode 100644
index 00000000000..2b985303339
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnectionConstants.java
@@ -0,0 +1,34 @@
+/*
+ * 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.ipc;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Constants to be used by RPC connection based utilities.
+ */
+@InterfaceAudience.Private
+public final class RpcConnectionConstants {
+
+  private RpcConnectionConstants() {
+  }
+
+  public static final String RELOGIN_IS_IN_PROGRESS =
+    "Can not send request because relogin is in progress.";
+
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index af22fba2729..abc9c575a62 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -22,11 +22,13 @@ import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import javax.security.sasl.SaslException;
 import org.apache.hadoop.hbase.CallQueueTooBigException;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.ipc.RpcConnectionConstants;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.ServerListener;
@@ -287,17 +289,15 @@ public class RSProcedureDispatcher extends RemoteProcedureDispatcher<MasterProce
           numberOfAttemptsSoFar);
         return false;
       }
-      // This exception is thrown in the rpc framework, where we can make sure that the call has not
-      // been executed yet, so it is safe to mark it as fail. Especially for open a region, we'd
-      // better choose another region server.
+      // This category of exceptions is thrown in the rpc framework, where we can make sure
+      // that the call has not been executed yet, so it is safe to mark it as fail.
+      // Especially for open a region, we'd better choose another region server.
       // Notice that, it is safe to quit only if this is the first time we send request to region
       // server. Maybe the region server has accepted our request the first time, and then there is
-      // a network error which prevents we receive the response, and the second time we hit a
-      // CallQueueTooBigException, obviously it is not safe to quit here, otherwise it may lead to a
-      // double assign...
-      if (e instanceof CallQueueTooBigException && numberOfAttemptsSoFar == 0) {
-        LOG.warn("request to {} failed due to {}, try={}, this usually because"
-          + " server is overloaded, give up", serverName, e.toString(), numberOfAttemptsSoFar);
+      // a network error which prevents we receive the response, and the second time we hit
+      // this category of exceptions, obviously it is not safe to quit here, otherwise it may lead
+      // to a double assign...
+      if (numberOfAttemptsSoFar == 0 && unableToConnectToServer(e)) {
         return false;
       }
       // Always retry for other exception types if the region server is not dead yet.
@@ -330,6 +330,47 @@ public class RSProcedureDispatcher extends RemoteProcedureDispatcher<MasterProce
       return true;
     }
 
+    /**
+     * The category of exceptions where we can ensure that the request has not yet been received
+     * and/or processed by the target regionserver yet and hence we can determine whether it is safe
+     * to choose different regionserver as the target.
+     * @param e IOException thrown by the underlying rpc framework.
+     * @return true if the exception belongs to the category where the regionserver has not yet
+     *         received the request yet.
+     */
+    private boolean unableToConnectToServer(IOException e) {
+      if (e instanceof CallQueueTooBigException) {
+        LOG.warn("request to {} failed due to {}, try={}, this usually because"
+          + " server is overloaded, give up", serverName, e, numberOfAttemptsSoFar);
+        return true;
+      }
+      if (isSaslError(e)) {
+        LOG.warn("{} is not reachable; give up after first attempt", serverName, e);
+        return true;
+      }
+      return false;
+    }
+
+    private boolean isSaslError(IOException e) {
+      Throwable cause = e;
+      while (true) {
+        if (cause instanceof IOException) {
+          IOException unwrappedCause = unwrapException((IOException) cause);
+          if (
+            unwrappedCause instanceof SaslException
+              || (unwrappedCause.getMessage() != null && unwrappedCause.getMessage()
+                .contains(RpcConnectionConstants.RELOGIN_IS_IN_PROGRESS))
+          ) {
+            return true;
+          }
+        }
+        cause = cause.getCause();
+        if (cause == null) {
+          return false;
+        }
+      }
+    }
+
     private long getMaxWaitTime() {
       if (this.maxWaitTime < 0) {
         // This is the max attempts, not retries, so it should be at least 1.