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 2020/04/26 02:25:02 UTC

[hbase] 01/02: Revert "HBASE-23381 Ensure Netty client receives at least one response before considering SASL negotiation complete"

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

commit c1f06344626e63f5494e41263a3f12ed9dea1002
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Sun Apr 26 10:22:46 2020 +0800

    Revert "HBASE-23381 Ensure Netty client receives at least one response before considering SASL negotiation complete"
    
    Incorrect jira issue number
    
    This reverts commit 78eecd3a7d291c46432efa4be16a04c2e3e0a027.
---
 .../hadoop/hbase/ipc/NettyRpcConnection.java       |   8 +-
 .../hbase/security/AbstractHBaseSaslRpcClient.java |  16 +--
 .../security/NettyHBaseSaslRpcClientHandler.java   |  23 +----
 .../ShadeSaslClientAuthenticationProvider.java     |   6 --
 .../ShadeSaslServerAuthenticationProvider.java     |   1 +
 .../TestShadeSaslAuthenticationProvider.java       | 109 ++++-----------------
 .../hadoop/hbase/ipc/ServerRpcConnection.java      |   2 +-
 7 files changed, 26 insertions(+), 139 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 eff4a13..7d91fd9 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
@@ -149,10 +149,6 @@ class NettyRpcConnection extends RpcConnection {
     if (error instanceof FallbackDisallowedException) {
       return;
     }
-    if (!provider.canRetry()) {
-      LOG.trace("SASL Provider does not support retries");
-      return;
-    }
     synchronized (this) {
       if (reloginInProgress) {
         return;
@@ -163,7 +159,9 @@ class NettyRpcConnection extends RpcConnection {
         @Override
         public void run() {
           try {
-            provider.relogin();
+            if (provider.canRetry()) {
+              provider.relogin();
+            }
           } catch (IOException e) {
             LOG.warn("Relogin failed", e);
           }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
index 26b3811..b1f0861 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/AbstractHBaseSaslRpcClient.java
@@ -88,24 +88,12 @@ public abstract class AbstractHBaseSaslRpcClient {
     }
   }
 
-  /**
-   * Computes the initial response a client sends to a server to begin the SASL
-   * challenge/response handshake. If the client's SASL mechanism does not require
-   * that an initial response is sent to begin the handshake, this method will return
-   * a null byte array, indicating no initial response needs to be sent by this client.
-   *
-   * It is unclear as to whether all SASL implementations will return a non-empty initial
-   * response, so this implementation is written such that this is allowed. All known
-   * SASL mechanism implementations in the JDK provide non-empty initial responses.
-   *
-   * @return The client's initial response to send the server (which may be empty), or null
-   *    if this implementation does not require an initial response to be sent.
-   */
   public byte[] getInitialResponse() throws SaslException {
     if (saslClient.hasInitialResponse()) {
       return saslClient.evaluateChallenge(EMPTY_TOKEN);
+    } else {
+      return EMPTY_TOKEN;
     }
-    return null;
   }
 
   public boolean isComplete() {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java
index aff3993..e011cc6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/NettyHBaseSaslRpcClientHandler.java
@@ -54,8 +54,6 @@ public class NettyHBaseSaslRpcClientHandler extends SimpleChannelInboundHandler<
 
   private final Configuration conf;
 
-  private final SaslClientAuthenticationProvider provider;
-
   // flag to mark if Crypto AES encryption is enable
   private boolean needProcessConnectionHeader = false;
 
@@ -70,7 +68,6 @@ public class NettyHBaseSaslRpcClientHandler extends SimpleChannelInboundHandler<
     this.saslPromise = saslPromise;
     this.ugi = ugi;
     this.conf = conf;
-    this.provider = provider;
     this.saslRpcClient = new NettyHBaseSaslRpcClient(conf, provider, token, serverAddr,
         securityInfo, fallbackAllowed, conf.get(
         "hbase.rpc.protection", SaslUtil.QualityOfProtection.AUTHENTICATION.name().toLowerCase()));
@@ -87,11 +84,6 @@ public class NettyHBaseSaslRpcClientHandler extends SimpleChannelInboundHandler<
       return;
     }
 
-    // HBASE-23881 Clearly log when the client thinks that the SASL negotiation is complete.
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("SASL negotiation for {} is complete", provider.getSaslAuthMethod().getName());
-    }
-
     saslRpcClient.setupSaslHandler(ctx.pipeline());
     setCryptoAESOption();
 
@@ -121,19 +113,8 @@ public class NettyHBaseSaslRpcClientHandler extends SimpleChannelInboundHandler<
       });
       if (initialResponse != null) {
         writeResponse(ctx, initialResponse);
-      } else {
-        LOG.trace("SASL initialResponse was null, not sending response to server.");
       }
-      // HBASE-23881 We do not want to check if the SaslClient thinks the handshake is
-      // complete as, at this point, we've not heard a back from the server with it's reply
-      // to our first challenge response. We should wait for at least one reply
-      // from the server before calling negotiation complete.
-      //
-      // Each SASL mechanism has its own handshake. Some mechanisms calculate a single client buffer
-      // to be sent to the server while others have multiple exchanges to negotiate authentication. GSSAPI(Kerberos)
-      // and DIGEST-MD5 both are examples of mechanisms which have multiple steps. Mechanisms which have multiple steps
-      // will not return true on `SaslClient#isComplete()` until the handshake has fully completed. Mechanisms which
-      // only send a single buffer may return true on `isComplete()` after that initial response is calculated.
+      tryComplete(ctx);
     } catch (Exception e) {
       // the exception thrown by handlerAdded will not be passed to the exceptionCaught below
       // because netty will remove a handler if handlerAdded throws an exception.
@@ -165,8 +146,6 @@ public class NettyHBaseSaslRpcClientHandler extends SimpleChannelInboundHandler<
     });
     if (response != null) {
       writeResponse(ctx, response);
-    } else {
-      LOG.trace("SASL challenge response was empty, not sending response to server.");
     }
     tryComplete(ctx);
   }
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/security/provider/example/ShadeSaslClientAuthenticationProvider.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/security/provider/example/ShadeSaslClientAuthenticationProvider.java
index 761a2f6..7cda97b 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/security/provider/example/ShadeSaslClientAuthenticationProvider.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/security/provider/example/ShadeSaslClientAuthenticationProvider.java
@@ -62,12 +62,6 @@ public class ShadeSaslClientAuthenticationProvider extends ShadeSaslAuthenticati
     return userInfoPB.build();
   }
 
-  @Override
-  public boolean canRetry() {
-    // A static username/password either works or it doesn't. No kind of relogin/retry necessary.
-    return false;
-  }
-
   static class ShadeSaslClientCallbackHandler implements CallbackHandler {
     private final String username;
     private final char[] password;
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/security/provider/example/ShadeSaslServerAuthenticationProvider.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/security/provider/example/ShadeSaslServerAuthenticationProvider.java
index 8bba8d6..dc8d89b 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/security/provider/example/ShadeSaslServerAuthenticationProvider.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/security/provider/example/ShadeSaslServerAuthenticationProvider.java
@@ -141,6 +141,7 @@ public class ShadeSaslServerAuthenticationProvider extends ShadeSaslAuthenticati
 
     @Override public void handle(Callback[] callbacks)
         throws InvalidToken, UnsupportedCallbackException {
+      LOG.info("SaslServerCallbackHandler called", new Exception());
       NameCallback nc = null;
       PasswordCallback pc = null;
       AuthorizeCallback ac = null;
diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/security/provider/example/TestShadeSaslAuthenticationProvider.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/security/provider/example/TestShadeSaslAuthenticationProvider.java
index 79e8c57..001842f 100644
--- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/security/provider/example/TestShadeSaslAuthenticationProvider.java
+++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/security/provider/example/TestShadeSaslAuthenticationProvider.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.security.provider.example;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -27,12 +26,8 @@ import java.io.BufferedWriter;
 import java.io.File;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
-import java.io.PrintWriter;
-import java.io.StringWriter;
 import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
 import java.util.Collections;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
@@ -42,6 +37,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -52,14 +48,11 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.MasterRegistry;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.exceptions.MasterRegistryFetchException;
 import org.apache.hadoop.hbase.security.HBaseKerberosUtils;
 import org.apache.hadoop.hbase.security.provider.SaslClientAuthenticationProviders;
 import org.apache.hadoop.hbase.security.provider.SaslServerAuthenticationProviders;
@@ -68,12 +61,8 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -82,18 +71,14 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 @Category({MediumTests.class, SecurityTests.class})
 public class TestShadeSaslAuthenticationProvider {
-  private static final Logger LOG = LoggerFactory.getLogger(TestShadeSaslAuthenticationProvider.class);
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestShadeSaslAuthenticationProvider.class);
 
-
   private static final char[] USER1_PASSWORD = "foobarbaz".toCharArray();
 
   static LocalHBaseCluster createCluster(HBaseTestingUtility util, File keytabFile,
@@ -235,84 +220,26 @@ public class TestShadeSaslAuthenticationProvider {
     }
   }
 
-  @Test
+  @Test(expected = DoNotRetryIOException.class)
   public void testNegativeAuthentication() throws Exception {
-    List<Pair<String, Class<? extends Exception>>> params = new ArrayList<>();
-    // Master-based connection will fail to ask the master its cluster ID
-    // as a part of creating the Connection.
-    params.add(new Pair<String, Class<? extends Exception>>(
-        MasterRegistry.class.getName(), MasterRegistryFetchException.class));
-    // ZK based connection will fail on the master RPC
-    params.add(new Pair<String, Class<? extends Exception>>(
-        // ZKConnectionRegistry is package-private
-        HConstants.ZK_CONNECTION_REGISTRY_CLASS, RetriesExhaustedException.class));
-
-    params.forEach((pair) -> {
-      LOG.info("Running negative authentication test for client registry {}, expecting {}",
-          pair.getFirst(), pair.getSecond().getName());
-      // Validate that we can read that record back out as the user with our custom auth'n
-      final Configuration clientConf = new Configuration(CONF);
-      clientConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
-      clientConf.set(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, pair.getFirst());
-      try (Connection conn = ConnectionFactory.createConnection(clientConf)) {
-        UserGroupInformation user1 = UserGroupInformation.createUserForTesting(
-            "user1", new String[0]);
-        user1.addToken(
-            ShadeClientTokenUtil.obtainToken(conn, "user1", "not a real password".toCharArray()));
-
-        LOG.info("Executing request to HBase Master which should fail");
-        user1.doAs(new PrivilegedExceptionAction<Void>() {
-          @Override public Void run() throws Exception {
-            try (Connection conn = ConnectionFactory.createConnection(clientConf);) {
-              conn.getAdmin().listTableDescriptors();
-              fail("Should not successfully authenticate with HBase");
-            } catch (Exception e) {
-              LOG.info("Caught exception in negative Master connectivity test", e);
-              assertEquals("Found unexpected exception", pair.getSecond(), e.getClass());
-              validateRootCause(Throwables.getRootCause(e));
-            }
-            return null;
-          }
-        });
-
-        LOG.info("Executing request to HBase RegionServer which should fail");
-        user1.doAs(new PrivilegedExceptionAction<Void>() {
-          @Override public Void run() throws Exception {
-            // A little contrived because, with MasterRegistry, we'll still fail on talking
-            // to the HBase master before trying to talk to a RegionServer.
-            try (Connection conn = ConnectionFactory.createConnection(clientConf);
-                Table t = conn.getTable(tableName)) {
-              t.get(new Get(Bytes.toBytes("r1")));
-              fail("Should not successfully authenticate with HBase");
-            } catch (Exception e) {
-              LOG.info("Caught exception in negative RegionServer connectivity test", e);
-              assertEquals("Found unexpected exception", pair.getSecond(), e.getClass());
-              validateRootCause(Throwables.getRootCause(e));
-            }
+    // Validate that we can read that record back out as the user with our custom auth'n
+    final Configuration clientConf = new Configuration(CONF);
+    clientConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
+    try (Connection conn = ConnectionFactory.createConnection(clientConf)) {
+      UserGroupInformation user1 = UserGroupInformation.createUserForTesting(
+          "user1", new String[0]);
+      user1.addToken(
+          ShadeClientTokenUtil.obtainToken(conn, "user1", "not a real password".toCharArray()));
+      user1.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override public Void run() throws Exception {
+          try (Connection conn = ConnectionFactory.createConnection(clientConf);
+              Table t = conn.getTable(tableName)) {
+            t.get(new Get(Bytes.toBytes("r1")));
+            fail("Should not successfully authenticate with HBase");
             return null;
           }
-        });
-      } catch (InterruptedException e) {
-        LOG.error("Caught interrupted exception", e);
-        Thread.currentThread().interrupt();
-        return;
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    });
-  }
-
-  void validateRootCause(Throwable rootCause) {
-    LOG.info("Root cause was", rootCause);
-    if (rootCause instanceof RemoteException) {
-      RemoteException re = (RemoteException) rootCause;
-      IOException actualException = re.unwrapRemoteException();
-      assertEquals(InvalidToken.class, actualException.getClass());
-    } else {
-      StringWriter writer = new StringWriter();
-      rootCause.printStackTrace(new PrintWriter(writer));
-      String text = writer.toString();
-      assertTrue("Message did not contain expected text", text.contains(InvalidToken.class.getName()));
+        }
+      });
     }
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java
index e55254e..d49a904 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java
@@ -373,7 +373,7 @@ abstract class ServerRpcConnection implements Closeable {
         String clientIP = this.toString();
         // attempting user could be null
         RpcServer.AUDITLOG
-            .warn("{}{}: {}", RpcServer.AUTH_FAILED_FOR, clientIP, saslServer.getAttemptingUser());
+            .warn("{} {}: {}", RpcServer.AUTH_FAILED_FOR, clientIP, saslServer.getAttemptingUser());
         throw e;
       }
       if (replyToken != null) {