You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2013/08/08 22:05:15 UTC

svn commit: r1512010 - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ conf/ jdbc/src/java/org/apache/hive/jdbc/ metastore/src/java/org/apache/hadoop/hive/metastore/ service/src/java/org/apache/hive/service/auth/ shims/src/common-secure/ja...

Author: hashutosh
Date: Thu Aug  8 20:05:15 2013
New Revision: 1512010

URL: http://svn.apache.org/r1512010
Log:
HIVE-4911 : Enable QOP configuration for Hive Server 2 thrift transport (Arup Malakar via Ashutosh Chauhan)

Added:
    hive/trunk/service/src/java/org/apache/hive/service/auth/SaslQOP.java
Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/conf/hive-default.xml.template
    hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
    hive/trunk/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java
    hive/trunk/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java
    hive/trunk/shims/src/common-secure/test/org/apache/hadoop/hive/thrift/TestHadoop20SAuthBridge.java
    hive/trunk/shims/src/common/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge.java

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1512010&r1=1512009&r2=1512010&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Thu Aug  8 20:05:15 2013
@@ -728,6 +728,7 @@ public class HiveConf extends Configurat
 
     HIVE_SERVER2_THRIFT_PORT("hive.server2.thrift.port", 10000),
     HIVE_SERVER2_THRIFT_BIND_HOST("hive.server2.thrift.bind.host", ""),
+    HIVE_SERVER2_THRIFT_SASL_QOP("hive.server2.thrift.sasl.qop", "auth"),
 
 
     // HiveServer2 auth configuration

Modified: hive/trunk/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/trunk/conf/hive-default.xml.template?rev=1512010&r1=1512009&r2=1512010&view=diff
==============================================================================
--- hive/trunk/conf/hive-default.xml.template (original)
+++ hive/trunk/conf/hive-default.xml.template Thu Aug  8 20:05:15 2013
@@ -1930,6 +1930,11 @@
   </description>
 </property>
 
+<property>
+  <name>hive.server2.thrift.sasl.qop</name>
+  <value>auth</auth>
+  <description>Sasl QOP value; one of 'auth', 'auth-int' and 'auth-conf'</description>
+</property>
 
 </configuration>
 

Modified: hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
URL: http://svn.apache.org/viewvc/hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java?rev=1512010&r1=1512009&r2=1512010&view=diff
==============================================================================
--- hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java (original)
+++ hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java Thu Aug  8 20:05:15 2013
@@ -33,6 +33,7 @@ import java.sql.SQLXML;
 import java.sql.Savepoint;
 import java.sql.Statement;
 import java.sql.Struct;
+import java.util.HashMap;
 import java.util.concurrent.Executor;
 import java.util.LinkedList;
 import java.util.List;
@@ -40,11 +41,13 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
 
+import javax.security.sasl.Sasl;
 import javax.security.sasl.SaslException;
 
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.service.auth.KerberosSaslHelper;
 import org.apache.hive.service.auth.PlainSaslHelper;
+import org.apache.hive.service.auth.SaslQOP;
 import org.apache.hive.service.cli.thrift.EmbeddedThriftCLIService;
 import org.apache.hive.service.cli.thrift.TCLIService;
 import org.apache.hive.service.cli.thrift.TCloseSessionReq;
@@ -65,6 +68,7 @@ import org.apache.thrift.transport.TTran
  */
 public class HiveConnection implements java.sql.Connection {
   private static final String HIVE_AUTH_TYPE= "auth";
+  private static final String HIVE_AUTH_QOP = "sasl.qop";
   private static final String HIVE_AUTH_SIMPLE = "noSasl";
   private static final String HIVE_AUTH_USER = "user";
   private static final String HIVE_AUTH_PRINCIPAL = "principal";
@@ -132,8 +136,19 @@ public class HiveConnection implements j
         || !sessConf.get(HIVE_AUTH_TYPE).equals(HIVE_AUTH_SIMPLE)){
       try {
         if (sessConf.containsKey(HIVE_AUTH_PRINCIPAL)) {
+          Map<String, String> saslProps = new HashMap<String, String>();
+          SaslQOP saslQOP = SaslQOP.AUTH;
+          if(sessConf.containsKey(HIVE_AUTH_QOP)) {
+            try {
+              saslQOP = SaslQOP.fromString(sessConf.get(HIVE_AUTH_QOP));
+            } catch (IllegalArgumentException e) {
+              throw new SQLException("Invalid " + HIVE_AUTH_QOP + " parameter. " + e.getMessage(), "42000", e);
+            }
+          }
+          saslProps.put(Sasl.QOP, saslQOP.toString());
+          saslProps.put(Sasl.SERVER_AUTH, "true");
           transport = KerberosSaslHelper.getKerberosTransport(
-                  sessConf.get(HIVE_AUTH_PRINCIPAL), host, transport);
+                  sessConf.get(HIVE_AUTH_PRINCIPAL), host, transport, saslProps);
         } else {
           String userName = sessConf.get(HIVE_AUTH_USER);
           if ((userName == null) || userName.isEmpty()) {

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java?rev=1512010&r1=1512009&r2=1512010&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java Thu Aug  8 20:05:15 2013
@@ -4376,7 +4376,8 @@ public class HiveMetaStore extends Thrif
         // start delegation token manager
         HMSHandler hmsHandler = new HMSHandler("new db based metaserver", conf);
         saslServer.startDelegationTokenSecretManager(conf, hmsHandler);
-        transFactory = saslServer.createTransportFactory();
+        transFactory = saslServer.createTransportFactory(
+                MetaStoreUtils.getMetaStoreSaslProperties(conf));
         processor = saslServer.wrapProcessor(
           new ThriftHiveMetastore.Processor<HMSHandler>(hmsHandler));
         LOG.info("Starting DB backed MetaStore Server in Secure Mode");

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java?rev=1512010&r1=1512009&r2=1512010&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java Thu Aug  8 20:05:15 2013
@@ -258,17 +258,17 @@ public class HiveMetaStoreClient impleme
               String tokenSig = conf.get("hive.metastore.token.signature");
               // tokenSig could be null
               tokenStrForm = shim.getTokenStrForm(tokenSig);
-
               if(tokenStrForm != null) {
                 // authenticate using delegation tokens via the "DIGEST" mechanism
                 transport = authBridge.createClientTransport(null, store.getHost(),
-                    "DIGEST", tokenStrForm, transport);
+                    "DIGEST", tokenStrForm, transport,
+                        MetaStoreUtils.getMetaStoreSaslProperties(conf));
               } else {
                 String principalConfig =
                     conf.getVar(HiveConf.ConfVars.METASTORE_KERBEROS_PRINCIPAL);
                 transport = authBridge.createClientTransport(
                     principalConfig, store.getHost(), "KERBEROS", null,
-                    transport);
+                    transport, MetaStoreUtils.getMetaStoreSaslProperties(conf));
               }
             } catch (IOException ioe) {
               LOG.error("Couldn't create client transport", ioe);

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java?rev=1512010&r1=1512009&r2=1512010&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java Thu Aug  8 20:05:15 2013
@@ -64,9 +64,9 @@ import org.apache.hadoop.hive.serde2.obj
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge;
 
-
 public class MetaStoreUtils {
 
   protected static final Log LOG = LogFactory.getLog("hive.log");
@@ -1264,6 +1264,18 @@ public class MetaStoreUtils {
     return true;
   }
 
+  /**
+   * Read and return the meta store Sasl configuration. Currently it uses the default
+   * Hadoop SASL configuration and can be configured using "hadoop.rpc.protection"
+   * @param conf
+   * @return The SASL configuration
+   */
+  public static Map<String, String> getMetaStoreSaslProperties(HiveConf conf) {
+    // As of now Hive Meta Store uses the same configuration as Hadoop SASL configuration
+    return ShimLoader.getHadoopThriftAuthBridge().getHadoopSaslProperties(conf);
+  }
+
+
   private static String getPartitionValWithInvalidCharacter(List<String> partVals,
       Pattern partitionValidationPattern) {
     if (partitionValidationPattern == null) {

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java?rev=1512010&r1=1512009&r2=1512010&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java Thu Aug  8 20:05:15 2013
@@ -20,6 +20,7 @@ package org.apache.hive.service.auth;
 import java.io.IOException;
 
 import javax.security.auth.login.LoginException;
+import javax.security.sasl.Sasl;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -29,8 +30,15 @@ import org.apache.hive.service.cli.thrif
 import org.apache.thrift.TProcessorFactory;
 import org.apache.thrift.transport.TTransportException;
 import org.apache.thrift.transport.TTransportFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.text.MessageFormat;
+import java.util.HashMap;
+import java.util.Map;
 
 public class HiveAuthFactory {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveAuthFactory.class);
 
   public static enum AuthTypes {
     NOSASL("NOSASL"),
@@ -71,13 +79,32 @@ public class HiveAuthFactory {
     }
   }
 
+  public Map<String, String> getSaslProperties() {
+    Map<String, String> saslProps = new HashMap<String, String>();
+    SaslQOP saslQOP =
+            SaslQOP.fromString(conf.getVar(ConfVars.HIVE_SERVER2_THRIFT_SASL_QOP));
+    // hadoop.rpc.protection being set to a higher level than hive.server2.thrift.rpc.protection
+    // does not make sense in most situations. Log warning message in such cases.
+    Map<String, String> hadoopSaslProps =  ShimLoader.getHadoopThriftAuthBridge().
+            getHadoopSaslProperties(conf);
+    SaslQOP hadoopSaslQOP = SaslQOP.fromString(hadoopSaslProps.get(Sasl.QOP));
+    if(hadoopSaslQOP.ordinal() > saslQOP.ordinal()) {
+      LOG.warn(MessageFormat.format("\"hadoop.rpc.protection\" is set to higher security level " +
+              "{0} then {1} which is set to {2}", hadoopSaslQOP.toString(),
+              ConfVars.HIVE_SERVER2_THRIFT_SASL_QOP.varname, saslQOP.toString()));
+    }
+    saslProps.put(Sasl.QOP, saslQOP.toString());
+    saslProps.put(Sasl.SERVER_AUTH, "true");
+    return saslProps;
+  }
+
   public TTransportFactory getAuthTransFactory() throws LoginException {
 
     TTransportFactory transportFactory;
 
     if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) {
       try {
-        transportFactory = saslServer.createTransportFactory();
+        transportFactory = saslServer.createTransportFactory(getSaslProperties());
       } catch (TTransportException e) {
         throw new LoginException(e.getMessage());
       }

Modified: hive/trunk/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java?rev=1512010&r1=1512009&r2=1512010&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/KerberosSaslHelper.java Thu Aug  8 20:05:15 2013
@@ -18,6 +18,7 @@
 package org.apache.hive.service.auth;
 
 import java.io.IOException;
+import java.util.Map;
 
 import javax.security.sasl.SaslException;
 
@@ -56,7 +57,7 @@ public class KerberosSaslHelper {
   }
 
   public static TTransport getKerberosTransport(String principal, String host,
-      final TTransport underlyingTransport) throws SaslException {
+      final TTransport underlyingTransport, Map<String, String> saslProps) throws SaslException {
     try {
       final String names[] = principal.split("[/@]");
       if (names.length != 3) {
@@ -67,7 +68,7 @@ public class KerberosSaslHelper {
       HadoopThriftAuthBridge.Client authBridge =
         ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos");
       return authBridge.createClientTransport(principal, host,
-          "KERBEROS", null, underlyingTransport);
+          "KERBEROS", null, underlyingTransport, saslProps);
     } catch (IOException e) {
       throw new SaslException("Failed to open client transport", e);
     }

Added: hive/trunk/service/src/java/org/apache/hive/service/auth/SaslQOP.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/SaslQOP.java?rev=1512010&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/SaslQOP.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/SaslQOP.java Thu Aug  8 20:05:15 2013
@@ -0,0 +1,61 @@
+/**
+ * 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.hive.service.auth;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Possible values of  SASL quality-of-protection value.
+ */
+public enum SaslQOP {
+  AUTH("auth"), // Authentication only.
+  AUTH_INT("auth-int"), // Authentication and integrity checking by using signatures.
+  AUTH_CONF("auth-conf"); // Authentication, integrity and confidentiality checking
+                          // by using signatures and encryption.
+
+  public final String saslQop;
+
+  private static final Map<String, SaslQOP> strToEnum
+          = new HashMap<String, SaslQOP>();
+  static {
+    for (SaslQOP SaslQOP : values())
+      strToEnum.put(SaslQOP.toString(), SaslQOP);
+  }
+
+  private SaslQOP(final String saslQop) {
+    this.saslQop = saslQop;
+  }
+
+  public String toString() {
+    return saslQop;
+  }
+
+  public static SaslQOP fromString(String str) {
+    if(str != null) {
+      str = str.toLowerCase();
+    }
+    SaslQOP saslQOP = strToEnum.get(str);
+    if(saslQOP == null) {
+      throw new IllegalArgumentException("Unknown auth type: " + str + " Allowed values are: "
+              + strToEnum.keySet());
+    }
+    return saslQOP;
+  }
+}

Modified: hive/trunk/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java?rev=1512010&r1=1512009&r2=1512010&view=diff
==============================================================================
--- hive/trunk/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java (original)
+++ hive/trunk/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java Thu Aug  8 20:05:15 2013
@@ -24,6 +24,7 @@ import java.net.InetAddress;
 import java.net.Socket;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
+import java.util.Map;
 
 import javax.security.auth.callback.Callback;
 import javax.security.auth.callback.CallbackHandler;
@@ -89,6 +90,19 @@ import org.apache.thrift.transport.TTran
      return new Server(keytabFile, principalConf);
    }
 
+   /**
+    * Read and return Hadoop SASL configuration which can be configured using
+    * "hadoop.rpc.protection"
+    * @param conf
+    * @return Hadoop SASL configuration
+    */
+   @Override
+   public Map<String, String> getHadoopSaslProperties(Configuration conf) {
+     // Initialize the SaslRpcServer to ensure QOP parameters are read from conf
+     SaslRpcServer.init(conf);
+     return SaslRpcServer.SASL_PROPS;
+   }
+
    public static class Client extends HadoopThriftAuthBridge.Client {
      /**
       * Create a client-side SASL transport that wraps an underlying transport.
@@ -97,13 +111,14 @@ import org.apache.thrift.transport.TTran
       *               supported.
       * @param serverPrincipal The Kerberos principal of the target server.
       * @param underlyingTransport The underlying transport mechanism, usually a TSocket.
+      * @param saslProps the sasl properties to create the client with
       */
 
      @Override
      public TTransport createClientTransport(
        String principalConfig, String host,
-        String methodStr, String tokenStrForm, TTransport underlyingTransport)
-       throws IOException {
+       String methodStr, String tokenStrForm, TTransport underlyingTransport,
+       Map<String, String> saslProps) throws IOException {
        AuthMethod method = AuthMethod.valueOf(AuthMethod.class, methodStr);
 
        TTransport saslTransport = null;
@@ -115,7 +130,7 @@ import org.apache.thrift.transport.TTran
             method.getMechanismName(),
             null,
             null, SaslRpcServer.SASL_DEFAULT_REALM,
-            SaslRpcServer.SASL_PROPS, new SaslClientCallbackHandler(t),
+            saslProps, new SaslClientCallbackHandler(t),
             underlyingTransport);
            return new TUGIAssumingTransport(saslTransport, UserGroupInformation.getCurrentUser());
 
@@ -132,7 +147,7 @@ import org.apache.thrift.transport.TTran
                method.getMechanismName(),
                null,
                names[0], names[1],
-               SaslRpcServer.SASL_PROPS, null,
+               saslProps, null,
                underlyingTransport);
              return new TUGIAssumingTransport(saslTransport, UserGroupInformation.getCurrentUser());
            } catch (SaslException se) {
@@ -140,7 +155,7 @@ import org.apache.thrift.transport.TTran
            }
 
          default:
-        throw new IOException("Unsupported authentication method: " + method);
+           throw new IOException("Unsupported authentication method: " + method);
        }
      }
     private static class SaslClientCallbackHandler implements CallbackHandler {
@@ -271,10 +286,11 @@ import org.apache.thrift.transport.TTran
       * can be passed as both the input and output transport factory when
       * instantiating a TThreadPoolServer, for example.
       *
+      * @param saslProps Map of SASL properties
       */
      @Override
-     public TTransportFactory createTransportFactory() throws TTransportException
-     {
+     public TTransportFactory createTransportFactory(Map<String, String> saslProps)
+             throws TTransportException {
        // Parse out the kerberos principal, host, realm.
        String kerberosName = realUgi.getUserName();
        final String names[] = SaslRpcServer.splitKerberosName(kerberosName);
@@ -286,11 +302,11 @@ import org.apache.thrift.transport.TTran
        transFactory.addServerDefinition(
          AuthMethod.KERBEROS.getMechanismName(),
          names[0], names[1],  // two parts of kerberos principal
-         SaslRpcServer.SASL_PROPS,
+         saslProps,
          new SaslRpcServer.SaslGssCallbackHandler());
        transFactory.addServerDefinition(AuthMethod.DIGEST.getMechanismName(),
           null, SaslRpcServer.SASL_DEFAULT_REALM,
-          SaslRpcServer.SASL_PROPS, new SaslDigestCallbackHandler(secretManager));
+          saslProps, new SaslDigestCallbackHandler(secretManager));
 
        return new TUGIAssumingTransportFactory(transFactory, realUgi);
      }

Modified: hive/trunk/shims/src/common-secure/test/org/apache/hadoop/hive/thrift/TestHadoop20SAuthBridge.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/src/common-secure/test/org/apache/hadoop/hive/thrift/TestHadoop20SAuthBridge.java?rev=1512010&r1=1512009&r2=1512010&view=diff
==============================================================================
--- hive/trunk/shims/src/common-secure/test/org/apache/hadoop/hive/thrift/TestHadoop20SAuthBridge.java (original)
+++ hive/trunk/shims/src/common-secure/test/org/apache/hadoop/hive/thrift/TestHadoop20SAuthBridge.java Thu Aug  8 20:05:15 2013
@@ -29,6 +29,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Enumeration;
 import java.util.List;
+import java.util.Map;
 
 import junit.framework.TestCase;
 
@@ -77,13 +78,13 @@ public class TestHadoop20SAuthBridge ext
         super();
       }
       @Override
-      public TTransportFactory createTransportFactory()
+      public TTransportFactory createTransportFactory(Map<String, String> saslProps)
       throws TTransportException {
         TSaslServerTransport.Factory transFactory =
           new TSaslServerTransport.Factory();
         transFactory.addServerDefinition(AuthMethod.DIGEST.getMechanismName(),
             null, SaslRpcServer.SASL_DEFAULT_REALM,
-            SaslRpcServer.SASL_PROPS,
+            saslProps,
             new SaslDigestCallbackHandler(secretManager));
 
         return new TUGIAssumingTransportFactory(transFactory, realUgi);

Modified: hive/trunk/shims/src/common/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/src/common/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge.java?rev=1512010&r1=1512009&r2=1512010&view=diff
==============================================================================
--- hive/trunk/shims/src/common/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge.java (original)
+++ hive/trunk/shims/src/common/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge.java Thu Aug  8 20:05:15 2013
@@ -20,6 +20,7 @@
 
  import java.io.IOException;
 import java.net.InetAddress;
+import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.thrift.TProcessor;
@@ -50,6 +51,18 @@ import org.apache.thrift.transport.TTran
    }
 
 
+  /**
+   * Read and return Hadoop SASL configuration which can be configured using
+   * "hadoop.rpc.protection"
+   *
+   * @param conf
+   * @return Hadoop SASL configuration
+   */
+   public Map<String, String> getHadoopSaslProperties(Configuration conf) {
+     throw new UnsupportedOperationException(
+       "The current version of Hadoop does not support Authentication");
+   }
+
    public static abstract class Client {
    /**
     *
@@ -65,13 +78,14 @@ import org.apache.thrift.transport.TTran
     * @throws IOException
     */
      public abstract TTransport createClientTransport(
-       String principalConfig, String host,
-       String methodStr,String tokenStrForm, TTransport underlyingTransport)
-       throws IOException;
+             String principalConfig, String host,
+             String methodStr, String tokenStrForm, TTransport underlyingTransport,
+             Map<String, String> saslProps)
+             throws IOException;
    }
 
    public static abstract class Server {
-     public abstract TTransportFactory createTransportFactory() throws TTransportException;
+     public abstract TTransportFactory createTransportFactory(Map<String, String> saslProps) throws TTransportException;
      public abstract TProcessor wrapProcessor(TProcessor processor);
      public abstract TProcessor wrapNonAssumingProcessor(TProcessor processor);
      public abstract InetAddress getRemoteAddress();