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 2012/12/18 01:55:05 UTC

svn commit: r1423238 - in /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase: master/HMaster.java master/HMasterCommandLine.java regionserver/HRegionServer.java zookeeper/HQuorumPeer.java zookeeper/ZKUtil.java

Author: stack
Date: Tue Dec 18 00:55:03 2012
New Revision: 1423238

URL: http://svn.apache.org/viewvc?rev=1423238&view=rev
Log:
HBASE-4791 Allow Secure Zookeeper JAAS configuration to be programmatically set (rather than only by reading JAAS configuration file)

Modified:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1423238&r1=1423237&r2=1423238&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Tue Dec 18 00:55:03 2012
@@ -367,6 +367,10 @@ Server {
     this.rsFatals = new MemoryBoundedLogMessageBuffer(
         conf.getLong("hbase.master.buffer.for.rs.fatals", 1*1024*1024));
 
+    // login the zookeeper client principal (if using security)
+    ZKUtil.loginClient(this.conf, "hbase.zookeeper.client.keytab.file",
+      "hbase.zookeeper.client.kerberos.principal", this.isa.getHostName());
+
     // initialize server principal (if using secure Hadoop)
     User.login(conf, "hbase.master.keytab.file",
       "hbase.master.kerberos.principal", this.isa.getHostName());

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java?rev=1423238&r1=1423237&r2=1423238&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java Tue Dec 18 00:55:03 2012
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.ServerCommandLine;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.zookeeper.KeeperException;
 
 @InterfaceAudience.Private
@@ -129,6 +130,11 @@ public class HMasterCommandLine extends 
               + HConstants.ZOOKEEPER_CLIENT_PORT);
         }
         zooKeeperCluster.setDefaultClientPort(zkClientPort);
+
+        // login the zookeeper server principal (if using security)
+        ZKUtil.loginServer(conf, "hbase.zookeeper.server.keytab.file",
+          "hbase.zookeeper.server.kerberos.principal", null);
+
         int clientPort = zooKeeperCluster.startup(zkDataPath);
         if (clientPort != zkClientPort) {
           String errorMsg = "Could not start ZK at requested port of " +

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1423238&r1=1423237&r2=1423238&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Tue Dec 18 00:55:03 2012
@@ -514,6 +514,10 @@ public class  HRegionServer implements C
     this.rpcServer.setQosFunction((qosFunction = new QosFunction()));
     this.startcode = System.currentTimeMillis();
 
+    // login the zookeeper client principal (if using security)
+    ZKUtil.loginClient(this.conf, "hbase.zookeeper.client.keytab.file",
+      "hbase.zookeeper.client.kerberos.principal", this.isa.getHostName());
+
     // login the server principal (if using secure Hadoop)
     User.login(this.conf, "hbase.regionserver.keytab.file",
       "hbase.regionserver.kerberos.principal", this.isa.getHostName());

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java?rev=1423238&r1=1423237&r2=1423238&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java Tue Dec 18 00:55:03 2012
@@ -64,6 +64,12 @@ public class HQuorumPeer {
       writeMyID(zkProperties);
       QuorumPeerConfig zkConfig = new QuorumPeerConfig();
       zkConfig.parseProperties(zkProperties);
+
+      // login the zookeeper server principal (if using security)
+      ZKUtil.loginServer(conf, "hbase.zookeeper.server.keytab.file",
+        "hbase.zookeeper.server.kerberos.principal",
+        zkConfig.getClientPortAddress().getHostName());
+
       runZKServer(zkConfig);
     } catch (Exception e) {
       e.printStackTrace();

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java?rev=1423238&r1=1423237&r2=1423238&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java Tue Dec 18 00:55:03 2012
@@ -23,10 +23,20 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.PrintWriter;
 import java.net.InetSocketAddress;
+import java.net.InetAddress;
 import java.net.Socket;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.security.auth.login.LoginException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
+
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.authentication.util.KerberosUtil;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
@@ -48,6 +58,8 @@ import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.apache.zookeeper.server.ZooKeeperSaslServer;
 
 /**
  * Internal HBase utility class for ZooKeeper.
@@ -110,6 +122,170 @@ public class ZKUtil {
         retry, retryIntervalMillis);
   }
 
+  /**
+   * Log in the current zookeeper server process using the given configuration
+   * keys for the credential file and login principal.
+   *
+   * <p><strong>This is only applicable when running on secure hbase</strong>
+   * On regular HBase (without security features), this will safely be ignored.
+   * </p>
+   *
+   * @param conf The configuration data to use
+   * @param keytabFileKey Property key used to configure the path to the credential file
+   * @param userNameKey Property key used to configure the login principal
+   * @param hostname Current hostname to use in any credentials
+   * @throws IOException underlying exception from SecurityUtil.login() call
+   */
+  public static void loginServer(Configuration conf, String keytabFileKey,
+      String userNameKey, String hostname) throws IOException {
+    login(conf, keytabFileKey, userNameKey, hostname,
+          ZooKeeperSaslServer.LOGIN_CONTEXT_NAME_KEY,
+          JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME);
+  }
+
+  /**
+   * Log in the current zookeeper client using the given configuration
+   * keys for the credential file and login principal.
+   *
+   * <p><strong>This is only applicable when running on secure hbase</strong>
+   * On regular HBase (without security features), this will safely be ignored.
+   * </p>
+   *
+   * @param conf The configuration data to use
+   * @param keytabFileKey Property key used to configure the path to the credential file
+   * @param userNameKey Property key used to configure the login principal
+   * @param hostname Current hostname to use in any credentials
+   * @throws IOException underlying exception from SecurityUtil.login() call
+   */
+  public static void loginClient(Configuration conf, String keytabFileKey,
+      String userNameKey, String hostname) throws IOException {
+    login(conf, keytabFileKey, userNameKey, hostname,
+          ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY,
+          JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME);
+  }
+
+  /**
+   * Log in the current process using the given configuration keys for the
+   * credential file and login principal.
+   *
+   * <p><strong>This is only applicable when running on secure hbase</strong>
+   * On regular HBase (without security features), this will safely be ignored.
+   * </p>
+   *
+   * @param conf The configuration data to use
+   * @param keytabFileKey Property key used to configure the path to the credential file
+   * @param userNameKey Property key used to configure the login principal
+   * @param hostname Current hostname to use in any credentials
+   * @param loginContextProperty property name to expose the entry name
+   * @param loginContextName jaas entry name
+   * @throws IOException underlying exception from SecurityUtil.login() call
+   */
+  private static void login(Configuration conf, String keytabFileKey,
+      String userNameKey, String hostname,
+      String loginContextProperty, String loginContextName)
+      throws IOException {
+    if (!isSecureZooKeeper(conf))
+      return;
+
+    // User has specified a jaas.conf, keep this one as the good one.
+    // HBASE_OPTS="-Djava.security.auth.login.config=jaas.conf"
+    if (System.getProperty("java.security.auth.login.config") != null)
+      return;
+
+    String keytabFilename = conf.get(keytabFileKey);
+    String principalConfig = conf.get(userNameKey, System.getProperty("user.name"));
+    String principalName = SecurityUtil.getServerPrincipal(principalConfig, hostname);
+
+    // Initialize the "jaas.conf" for keyTab/principal,
+    // If keyTab is not specified use the Ticket Cache.
+    // and set the zookeeper login context name.
+    JaasConfiguration jaasConf = new JaasConfiguration(loginContextName,
+      keytabFilename, principalName);
+    javax.security.auth.login.Configuration.setConfiguration(jaasConf);
+    System.setProperty(loginContextProperty, loginContextName);
+  }
+
+  /**
+   * A JAAS configuration that defines the login modules that we want to use for login.
+   */
+  private static class JaasConfiguration extends javax.security.auth.login.Configuration {
+    private static final String SERVER_KEYTAB_KERBEROS_CONFIG_NAME =
+      "zookeeper-server-keytab-kerberos";
+    private static final String CLIENT_KEYTAB_KERBEROS_CONFIG_NAME =
+      "zookeeper-client-keytab-kerberos";
+
+    private static final Map<String, String> BASIC_JAAS_OPTIONS =
+      new HashMap<String,String>();
+    static {
+      String jaasEnvVar = System.getenv("HBASE_JAAS_DEBUG");
+      if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) {
+        BASIC_JAAS_OPTIONS.put("debug", "true");
+      }
+    }
+
+    private static final Map<String,String> KEYTAB_KERBEROS_OPTIONS =
+      new HashMap<String,String>();
+    static {
+      KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true");
+      KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true");
+      KEYTAB_KERBEROS_OPTIONS.put("refreshKrb5Config", "true");
+      KEYTAB_KERBEROS_OPTIONS.putAll(BASIC_JAAS_OPTIONS);
+    }
+
+    private static final AppConfigurationEntry KEYTAB_KERBEROS_LOGIN =
+      new AppConfigurationEntry(KerberosUtil.getKrb5LoginModuleName(),
+                                LoginModuleControlFlag.REQUIRED,
+                                KEYTAB_KERBEROS_OPTIONS);
+
+    private static final AppConfigurationEntry[] KEYTAB_KERBEROS_CONF =
+      new AppConfigurationEntry[]{KEYTAB_KERBEROS_LOGIN};
+
+    private javax.security.auth.login.Configuration baseConfig;
+    private final String loginContextName;
+    private final boolean useTicketCache;
+    private final String keytabFile;
+    private final String principal;
+
+    public JaasConfiguration(String loginContextName, String principal) {
+      this(loginContextName, principal, null, true);
+    }
+
+    public JaasConfiguration(String loginContextName, String principal, String keytabFile) {
+      this(loginContextName, principal, keytabFile, keytabFile == null || keytabFile.length() == 0);
+    }
+
+    private JaasConfiguration(String loginContextName, String principal,
+                             String keytabFile, boolean useTicketCache) {
+      try {
+        this.baseConfig = javax.security.auth.login.Configuration.getConfiguration();
+      } catch (SecurityException e) {
+        this.baseConfig = null;
+      }
+      this.loginContextName = loginContextName;
+      this.useTicketCache = useTicketCache;
+      this.keytabFile = keytabFile;
+      this.principal = principal;
+      LOG.info("JaasConfiguration loginContextName=" + loginContextName +
+               " principal=" + principal + " useTicketCache=" + useTicketCache +
+               " keytabFile=" + keytabFile);
+    }
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String appName) {
+      if (loginContextName.equals(appName)) {
+        if (!useTicketCache) {
+          KEYTAB_KERBEROS_OPTIONS.put("keyTab", keytabFile);
+          KEYTAB_KERBEROS_OPTIONS.put("useKeyTab", "true");
+        }
+        KEYTAB_KERBEROS_OPTIONS.put("principal", principal);
+        KEYTAB_KERBEROS_OPTIONS.put("useTicketCache", useTicketCache ? "true" : "false");
+        return KEYTAB_KERBEROS_CONF;
+      }
+      if (baseConfig != null) return baseConfig.getAppConfigurationEntry(appName);
+      return(null);
+    }
+  }
+
   //
   // Helper methods
   //
@@ -708,16 +884,20 @@ public class ZKUtil {
     setData(zkw, znode, data, -1);
   }
 
+  /**
+   * Returns whether or not secure authentication is enabled
+   * (whether <code>hbase.security.authentication</code> is set to
+   * <code>kerberos</code>.
+   */
   public static boolean isSecureZooKeeper(Configuration conf) {
-    // TODO: We need a better check for security enabled ZooKeeper. Currently
-    // the secure ZooKeeper client is set up using a supplied JaaS
-    // configuration file. But if the system property for the JaaS
-    // configuration file is set, this may not be an exclusive indication
-    // that HBase should set ACLs on znodes. As an alternative, we could do
-    // this more like Hadoop and build a JaaS configuration programmatically
-    // based on a site conf setting. The scope of such a change will be
-    // addressed in HBASE-4791.
-    return (System.getProperty("java.security.auth.login.config") != null);
+    // hbase shell need to use:
+    //    -Djava.security.auth.login.config=user-jaas.conf
+    // since each user has a different jaas.conf
+    if (System.getProperty("java.security.auth.login.config") != null)
+      return true;
+
+    // Master & RSs uses hbase.zookeeper.client.*
+    return "kerberos".equalsIgnoreCase(conf.get("hbase.security.authentication"));
   }
 
   private static ArrayList<ACL> createACL(ZooKeeperWatcher zkw, String node) {