You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2021/03/17 04:06:25 UTC

[hbase] branch branch-2.4 updated: HBASE-25665 Option to use hostname instead of canonical hostname for secure HBase cluster connection (#3051)

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

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


The following commit(s) were added to refs/heads/branch-2.4 by this push:
     new 90c147f  HBASE-25665 Option to use hostname instead of canonical hostname for secure HBase cluster connection (#3051)
90c147f is described below

commit 90c147f97c3f9fcfd54dbebe8c22542a0ae6d93e
Author: bitterfox <bi...@gmail.com>
AuthorDate: Wed Mar 17 13:04:25 2021 +0900

    HBASE-25665 Option to use hostname instead of canonical hostname for secure HBase cluster connection (#3051)
---
 .../GssSaslClientAuthenticationProvider.java       |  30 ++++-
 .../hadoop/hbase/security/SecurityConstants.java   |  14 ++-
 hbase-common/src/main/resources/hbase-default.xml  |   8 ++
 .../hadoop/hbase/security/TestSecureIPC.java       | 124 +++++++++++++++++++++
 4 files changed, 173 insertions(+), 3 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/GssSaslClientAuthenticationProvider.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/GssSaslClientAuthenticationProvider.java
index 2db865d..21a4828 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/GssSaslClientAuthenticationProvider.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/GssSaslClientAuthenticationProvider.java
@@ -26,6 +26,7 @@ import javax.security.sasl.SaslClient;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.security.SaslUtil;
+import org.apache.hadoop.hbase.security.SecurityConstants;
 import org.apache.hadoop.hbase.security.SecurityInfo;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.security.SecurityUtil;
@@ -44,15 +45,40 @@ public class GssSaslClientAuthenticationProvider extends GssSaslAuthenticationPr
   private static final Logger LOG = LoggerFactory.getLogger(
       GssSaslClientAuthenticationProvider.class);
 
+  private static boolean useCanonicalHostname(Configuration conf) {
+    return !conf.getBoolean(
+      SecurityConstants.UNSAFE_HBASE_CLIENT_KERBEROS_HOSTNAME_DISABLE_REVERSEDNS,
+      SecurityConstants.DEFAULT_UNSAFE_HBASE_CLIENT_KERBEROS_HOSTNAME_DISABLE_REVERSEDNS);
+  }
+
+  public static String getHostnameForServerPrincipal(Configuration conf, InetAddress addr) {
+    final String hostname;
+
+    if (useCanonicalHostname(conf)) {
+      hostname = addr.getCanonicalHostName();
+      if (hostname.equals(addr.getHostAddress())) {
+        LOG.warn("Canonical hostname for SASL principal is the same with IP address: "
+          + hostname + ", " + addr.getHostName() + ". Check DNS configuration or consider "
+          + SecurityConstants.UNSAFE_HBASE_CLIENT_KERBEROS_HOSTNAME_DISABLE_REVERSEDNS
+          + "=true");
+      }
+    } else {
+      hostname = addr.getHostName();
+    }
+
+    return hostname.toLowerCase();
+  }
+
   String getServerPrincipal(Configuration conf, SecurityInfo securityInfo, InetAddress server)
       throws IOException {
+    String hostname = getHostnameForServerPrincipal(conf, server);
+
     String serverKey = securityInfo.getServerPrincipal();
     if (serverKey == null) {
       throw new IllegalArgumentException(
           "Can't obtain server Kerberos config key from SecurityInfo");
     }
-    return SecurityUtil.getServerPrincipal(conf.get(serverKey),
-        server.getCanonicalHostName().toLowerCase());
+    return SecurityUtil.getServerPrincipal(conf.get(serverKey), hostname);
   }
 
   @Override
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/SecurityConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/SecurityConstants.java
index b5540d8..3e387e8 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/SecurityConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/SecurityConstants.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.security;
 
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -34,7 +35,18 @@ public final class SecurityConstants {
   public static final String REGIONSERVER_KRB_PRINCIPAL = "hbase.regionserver.kerberos.principal";
   public static final String REGIONSERVER_KRB_KEYTAB_FILE = "hbase.regionserver.keytab.file";
 
+  /**
+   * This config is for experts: don't set its value unless you really know what you are doing.
+   * When set to true, HBase client using SASL Kerberos will skip reverse DNS lookup and use provided
+   * hostname of the destination for the principal instead. See https://issues.apache.org/jira/browse/HBASE-25665
+   * for more details.
+   */
+  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+  public static final String UNSAFE_HBASE_CLIENT_KERBEROS_HOSTNAME_DISABLE_REVERSEDNS =
+    "hbase.unsafe.client.kerberos.hostname.disable.reversedns";
+  public static final boolean DEFAULT_UNSAFE_HBASE_CLIENT_KERBEROS_HOSTNAME_DISABLE_REVERSEDNS = false;
+
   private SecurityConstants() {
     // Can't be instantiated with this ctor.
   }
-}
\ No newline at end of file
+}
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index df94c5b..bf83a1f 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -1197,6 +1197,14 @@ possible configurations would overwhelm and obscure the important.
       MUST BE DISABLED for secure operation.</description>
   </property>
   <property>
+    <name>hbase.unsafe.client.kerberos.hostname.disable.reversedns</name>
+    <value>false</value>
+    <description>This config is for experts: don't set its value unless you really know what you are doing.
+      When set to true, HBase client using SASL Kerberos will skip reverse DNS lookup and use provided
+      hostname of the destination for the principal instead. See https://issues.apache.org/jira/browse/HBASE-25665
+      for more details.</description>
+  </property>
+  <property>
     <name>hbase.display.keys</name>
     <value>true</value>
     <description>When this is set to true the webUI and such will display all start/end keys
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java
index 9588699..09f21d7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureIPC.java
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlocking
 import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getKeytabFileForTesting;
 import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getPrincipalForTesting;
 import static org.apache.hadoop.hbase.security.HBaseKerberosUtils.getSecuredConfiguration;
+import static org.apache.hadoop.hbase.security.provider.SaslClientAuthenticationProviders.SELECTOR_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotSame;
 import static org.junit.Assert.assertSame;
@@ -29,12 +30,16 @@ import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
+import java.lang.reflect.Field;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
+import javax.security.sasl.SaslClient;
 import javax.security.sasl.SaslException;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -52,11 +57,18 @@ import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.ipc.RpcServerFactory;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.ipc.SimpleRpcServer;
+import org.apache.hadoop.hbase.security.provider.AuthenticationProviderSelector;
+import org.apache.hadoop.hbase.security.provider.BuiltInProviderSelector;
+import org.apache.hadoop.hbase.security.provider.SaslAuthMethod;
+import org.apache.hadoop.hbase.security.provider.SaslClientAuthenticationProvider;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -76,6 +88,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
 
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos;
 import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation;
 
 @RunWith(Parameterized.class)
 @Category({ SecurityTests.class, LargeTests.class })
@@ -165,6 +178,117 @@ public class TestSecureIPC {
   }
 
   @Test
+  public void testRpcCallWithEnabledKerberosSaslAuth_CanonicalHostname() throws Exception {
+    UserGroupInformation ugi2 = UserGroupInformation.getCurrentUser();
+
+    // check that the login user is okay:
+    assertSame(ugi2, ugi);
+    assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod());
+    assertEquals(krbPrincipal, ugi.getUserName());
+
+    enableCanonicalHostnameTesting(clientConf, "localhost");
+    clientConf.setBoolean(
+      SecurityConstants.UNSAFE_HBASE_CLIENT_KERBEROS_HOSTNAME_DISABLE_REVERSEDNS, false);
+    clientConf.set(HBaseKerberosUtils.KRB_PRINCIPAL, "hbase/_HOST@" + KDC.getRealm());
+
+    callRpcService(User.create(ugi2));
+  }
+
+  @Test
+  public void testRpcCallWithEnabledKerberosSaslAuth_NoCanonicalHostname() throws Exception {
+    UserGroupInformation ugi2 = UserGroupInformation.getCurrentUser();
+
+    // check that the login user is okay:
+    assertSame(ugi2, ugi);
+    assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod());
+    assertEquals(krbPrincipal, ugi.getUserName());
+
+    enableCanonicalHostnameTesting(clientConf, "127.0.0.1");
+    clientConf.setBoolean(
+      SecurityConstants.UNSAFE_HBASE_CLIENT_KERBEROS_HOSTNAME_DISABLE_REVERSEDNS, true);
+    clientConf.set(HBaseKerberosUtils.KRB_PRINCIPAL, "hbase/_HOST@" + KDC.getRealm());
+
+    callRpcService(User.create(ugi2));
+  }
+
+  private static void enableCanonicalHostnameTesting(Configuration conf, String canonicalHostname) {
+    conf.setClass(SELECTOR_KEY,
+      CanonicalHostnameTestingAuthenticationProviderSelector.class,
+      AuthenticationProviderSelector.class);
+    conf.set(CanonicalHostnameTestingAuthenticationProviderSelector.CANONICAL_HOST_NAME_KEY,
+      canonicalHostname);
+  }
+
+  public static class CanonicalHostnameTestingAuthenticationProviderSelector extends
+    BuiltInProviderSelector {
+    private static final String CANONICAL_HOST_NAME_KEY =
+      "CanonicalHostnameTestingAuthenticationProviderSelector.canonicalHostName";
+
+    @Override
+    public Pair<SaslClientAuthenticationProvider, Token<? extends TokenIdentifier>> selectProvider(
+      String clusterId, User user) {
+      final Pair<SaslClientAuthenticationProvider, Token<? extends TokenIdentifier>> pair =
+        super.selectProvider(clusterId, user);
+      pair.setFirst(createCanonicalHostNameTestingProvider(pair.getFirst()));
+      return pair;
+    }
+
+    SaslClientAuthenticationProvider createCanonicalHostNameTestingProvider(
+      SaslClientAuthenticationProvider delegate) {
+      return new SaslClientAuthenticationProvider() {
+        @Override
+        public SaslClient createClient(Configuration conf, InetAddress serverAddr,
+          SecurityInfo securityInfo, Token<? extends TokenIdentifier> token,
+          boolean fallbackAllowed, Map<String, String> saslProps) throws IOException {
+          final String s =
+            conf.get(CANONICAL_HOST_NAME_KEY);
+          if (s != null) {
+            try {
+              final Field canonicalHostName = InetAddress.class.getDeclaredField("canonicalHostName");
+              canonicalHostName.setAccessible(true);
+              canonicalHostName.set(serverAddr, s);
+            } catch (NoSuchFieldException | IllegalAccessException e) {
+              throw new RuntimeException(e);
+            }
+          }
+
+          return delegate.createClient(conf, serverAddr, securityInfo, token, fallbackAllowed, saslProps);
+        }
+
+        @Override
+        public UserInformation getUserInfo(User user) {
+          return delegate.getUserInfo(user);
+        }
+
+        @Override
+        public UserGroupInformation getRealUser(User ugi) {
+          return delegate.getRealUser(ugi);
+        }
+
+        @Override
+        public boolean canRetry() {
+          return delegate.canRetry();
+        }
+
+        @Override
+        public void relogin() throws IOException {
+          delegate.relogin();
+        }
+
+        @Override
+        public SaslAuthMethod getSaslAuthMethod() {
+          return delegate.getSaslAuthMethod();
+        }
+
+        @Override
+        public String getTokenKind() {
+          return delegate.getTokenKind();
+        }
+      };
+    }
+  }
+
+  @Test
   public void testRpcFallbackToSimpleAuth() throws Exception {
     String clientUsername = "testuser";
     UserGroupInformation clientUgi = UserGroupInformation.createUserForTesting(clientUsername,