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 2014/03/12 06:10:31 UTC

svn commit: r1576584 - in /hive/branches/branch-0.13: ./ itests/hive-unit/ itests/hive-unit/src/test/java/org/apache/hive/jdbc/ itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/ service/src/java/org/apache/hive/service/auth/

Author: hashutosh
Date: Wed Mar 12 05:10:30 2014
New Revision: 1576584

URL: http://svn.apache.org/r1576584
Log:
HIVE-6558 : HiveServer2 Plain SASL authentication broken after hadoop 2.3 upgrade (Prasad Mujumdar via Ashutosh Chauhan)

Added:
    hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniMr.java
Modified:
    hive/branches/branch-0.13/itests/hive-unit/pom.xml
    hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
    hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
    hive/branches/branch-0.13/pom.xml
    hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/PlainSaslHelper.java
    hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/PlainSaslServer.java

Modified: hive/branches/branch-0.13/itests/hive-unit/pom.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/itests/hive-unit/pom.xml?rev=1576584&r1=1576583&r2=1576584&view=diff
==============================================================================
--- hive/branches/branch-0.13/itests/hive-unit/pom.xml (original)
+++ hive/branches/branch-0.13/itests/hive-unit/pom.xml Wed Mar 12 05:10:30 2014
@@ -192,9 +192,20 @@
           <classifier>tests</classifier>
           <scope>test</scope>
         </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-tools</artifactId>
+          <version>${hadoop-20S.version}</version>
+          <scope>test</scope>
+        </dependency>
+        <dependency>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-servlet</artifactId>
+          <scope>test</scope>
+        </dependency>
       </dependencies>
     </profile>
-   <profile>
+    <profile>
       <id>hadoop-2</id>
       <dependencies>
         <dependency>
@@ -229,6 +240,16 @@
           <version>${hbase.hadoop2.version}</version>
           <scope>test</scope>
         </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-minicluster</artifactId>
+          <scope>test</scope>
+        </dependency>
+        <dependency>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-servlet</artifactId>
+          <scope>test</scope>
+        </dependency>
       </dependencies>
     </profile>
   </profiles>

Added: hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniMr.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniMr.java?rev=1576584&view=auto
==============================================================================
--- hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniMr.java (added)
+++ hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniMr.java Wed Mar 12 05:10:30 2014
@@ -0,0 +1,192 @@
+/**
+ * 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.jdbc;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Map;
+import java.util.HashMap;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.jdbc.miniHS2.MiniHS2;
+import org.apache.hive.service.cli.HiveSQLException;
+import org.apache.hive.service.cli.session.HiveSessionHook;
+import org.apache.hive.service.cli.session.HiveSessionHookContext;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestJdbcWithMiniMr {
+  public static final String TEST_TAG = "miniHS2.miniMr.tag";
+  public static final String TEST_TAG_VALUE = "miniHS2.miniMr.value";
+  public static class MiniMrTestSessionHook implements HiveSessionHook {
+     @Override
+     public void run(HiveSessionHookContext sessionHookContext) throws HiveSQLException {
+       sessionHookContext.getSessionConf().set(TEST_TAG, TEST_TAG_VALUE);
+     }
+   }
+
+  private static MiniHS2 miniHS2 = null;
+  private static Path dataFilePath;
+  private static String  dbName = "mrTestDb";
+  private Connection hs2Conn = null;
+  private Statement stmt;
+
+  @BeforeClass
+  public static void beforeTest() throws Exception {
+    Class.forName(MiniHS2.getJdbcDriverName());
+    HiveConf conf = new HiveConf();
+    conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    String dataFileDir = conf.get("test.data.files").replace('\\', '/')
+        .replace("c:", "");
+    dataFilePath = new Path(dataFileDir, "kv1.txt");
+    DriverManager.setLoginTimeout(0);
+    conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    miniHS2 = new MiniHS2(conf, true);
+    Map<String, String> overlayProps = new HashMap<String, String>();
+    overlayProps.put(ConfVars.HIVE_SERVER2_SESSION_HOOK.varname,
+        MiniMrTestSessionHook.class.getName());
+    miniHS2.start(overlayProps);
+    createDb();
+  }
+
+  // setup DB
+  private static void createDb() throws SQLException {
+    Connection conn = DriverManager.
+        getConnection(miniHS2.getJdbcURL(), System.getProperty("user.name"), "bar");
+    Statement stmt2 = conn.createStatement();
+    stmt2.execute("DROP DATABASE IF EXISTS " + dbName + " CASCADE");
+    stmt2.execute("CREATE DATABASE " + dbName);
+    stmt2.close();
+    conn.close();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL(dbName),
+            System.getProperty("user.name"), "bar");
+    stmt = hs2Conn.createStatement();
+    stmt.execute("USE " + dbName);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (hs2Conn != null) {
+      hs2Conn.close();
+    }
+  }
+
+  @AfterClass
+  public static void afterTest() throws Exception {
+    if (miniHS2 != null && miniHS2.isStarted()) {
+      miniHS2.stop();
+    }
+  }
+
+  /**
+   * Verify that the connection to HS2 with MiniMr is successful
+   * @throws Exception
+   */
+  @Test
+  public void testConnection() throws Exception {
+    // the session hook should set the property
+    verifyProperty(TEST_TAG, TEST_TAG_VALUE);
+  }
+
+  /**
+   * Run nonMr query
+   * @throws Exception
+   */
+  @Test
+  public void testNonMrQuery() throws Exception {
+    String tableName = "testTab1";
+    String resultVal = "val_238";
+    String queryStr = "SELECT * FROM " + tableName;
+
+    testKvQuery(tableName, queryStr, resultVal);
+  }
+
+  /**
+   * Run nonMr query
+   * @throws Exception
+   */
+  @Test
+  public void testMrQuery() throws Exception {
+    String tableName = "testTab2";
+    String resultVal = "val_238";
+    String queryStr = "SELECT * FROM " + tableName +
+        " where value = '" + resultVal + "'";
+
+    testKvQuery(tableName, queryStr, resultVal);
+  }
+
+  /**
+   * Verify if the given property contains the expected value
+   * @param propertyName
+   * @param expectedValue
+   * @throws Exception
+   */
+  private void verifyProperty(String propertyName, String expectedValue) throws Exception {
+    Statement stmt = hs2Conn .createStatement();
+    ResultSet res = stmt.executeQuery("set " + propertyName);
+    assertTrue(res.next());
+    String results[] = res.getString(1).split("=");
+    assertEquals("Property should be set", results.length, 2);
+    assertEquals("Property should be set", expectedValue, results[1]);
+  }
+
+  // create tables, verify query
+  private void testKvQuery(String tableName, String queryStr, String resultVal)
+      throws SQLException {
+    setupKv1Tabs(tableName);
+    verifyResult(queryStr, resultVal, 2);
+    stmt.execute("DROP TABLE " + tableName);
+  }
+
+  // create table and pupulate with kv1.txt
+  private void setupKv1Tabs(String tableName) throws SQLException {
+    Statement stmt = hs2Conn.createStatement();
+    // create table
+    stmt.execute("CREATE TABLE " + tableName
+        + " (under_col INT COMMENT 'the under column', value STRING)"
+        + " COMMENT ' test table'");
+
+    // load data
+    stmt.execute("load data local inpath '"
+        + dataFilePath.toString() + "' into table " + tableName);
+  }
+
+  // run given query and validate expecated result
+  private void verifyResult(String queryStr, String expString, int colPos)
+      throws SQLException {
+    ResultSet res = stmt.executeQuery(queryStr);
+    assertTrue(res.next());
+    assertEquals(expString, res.getString(colPos));
+    res.close();
+  }
+}

Modified: hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java?rev=1576584&r1=1576583&r2=1576584&view=diff
==============================================================================
--- hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java (original)
+++ hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java Wed Mar 12 05:10:30 2014
@@ -18,6 +18,9 @@
 
 package org.apache.hive.jdbc.miniHS2;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -33,6 +36,7 @@ public abstract class AbstractHiveServic
   private int binaryPort;
   private int httpPort;
   private boolean startedHiveService = false;
+  private List<String> addedProperties = new ArrayList<String>();
 
   public AbstractHiveService(HiveConf hiveConf, String hostname, int binaryPort, int httpPort) {
     this.hiveConf = hiveConf;
@@ -66,6 +70,17 @@ public abstract class AbstractHiveServic
   public void setConfProperty(String propertyKey, String propertyValue) {
     System.setProperty(propertyKey, propertyValue);
     hiveConf.set(propertyKey, propertyValue);
+    addedProperties.add(propertyKey);
+  }
+
+  /**
+   * Create system properties set by this server instance. This ensures that
+   * the changes made by current test are not impacting subsequent tests.
+   */
+  public void clearProperties() {
+    for (String propKey : addedProperties ) {
+      System.clearProperty(propKey);
+    }
   }
 
   /**

Modified: hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java?rev=1576584&r1=1576583&r2=1576584&view=diff
==============================================================================
--- hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java (original)
+++ hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java Wed Mar 12 05:10:30 2014
@@ -18,6 +18,8 @@
 
 package org.apache.hive.jdbc.miniHS2;
 
+import static org.junit.Assert.assertNotNull;
+
 import java.io.File;
 import java.io.IOException;
 import java.util.Map;
@@ -25,10 +27,15 @@ import java.util.concurrent.TimeoutExcep
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.HiveMetaStore;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.shims.HadoopShims.MiniDFSShim;
+import org.apache.hadoop.hive.shims.HadoopShims.MiniMrShim;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hive.service.Service;
 import org.apache.hive.service.cli.CLIServiceClient;
 import org.apache.hive.service.cli.SessionHandle;
@@ -43,23 +50,55 @@ public class MiniHS2 extends AbstractHiv
   private static final String driverName = "org.apache.hive.jdbc.HiveDriver";
   private HiveServer2 hiveServer2 = null;
   private final File baseDir;
+  private final Path baseDfsDir;
   private static final AtomicLong hs2Counter = new AtomicLong();
   private static final String HS2_BINARY_MODE = "binary";
   private static final String HS2_HTTP_MODE = "http";
+  private MiniMrShim mr;
+  private MiniDFSShim dfs;
 
   public MiniHS2(HiveConf hiveConf) throws IOException {
+    this(hiveConf, false);
+  }
+
+  public MiniHS2(HiveConf hiveConf, boolean useMiniMR) throws IOException {
     super(hiveConf, "localhost", MetaStoreUtils.findFreePort(), MetaStoreUtils.findFreePort());
     baseDir =  Files.createTempDir();
-    setWareHouseDir("file://" + baseDir.getPath() + File.separator + "warehouse");
+    FileSystem fs;
+    if (useMiniMR) {
+      dfs = ShimLoader.getHadoopShims().getMiniDfs(hiveConf, 4, true, null);
+      fs = dfs.getFileSystem();
+      mr = ShimLoader.getHadoopShims().getMiniMrCluster(hiveConf, 4,
+          fs.getUri().toString(), 1);
+      // store the config in system properties
+      mr.setupConfiguration(getHiveConf());
+      baseDfsDir =  new Path(new Path(fs.getUri()), "/base");
+    } else {
+      fs = FileSystem.getLocal(hiveConf);
+      baseDfsDir = new Path("file://"+ baseDir.getPath());
+    }
     String metaStoreURL =  "jdbc:derby:" + baseDir.getAbsolutePath() + File.separator + "test_metastore-" +
         hs2Counter.incrementAndGet() + ";create=true";
+
+    fs.mkdirs(baseDfsDir);
+    Path wareHouseDir = new Path(baseDfsDir, "warehouse");
+    fs.mkdirs(wareHouseDir);
+    setWareHouseDir(wareHouseDir.toString());
     System.setProperty(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, metaStoreURL);
     hiveConf.setVar(HiveConf.ConfVars.METASTORECONNECTURLKEY, metaStoreURL);
+    // reassign a new port, just in case if one of the MR services grabbed the last one
+    setBinaryPort(MetaStoreUtils.findFreePort());
     hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, HS2_BINARY_MODE);
     hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, getHost());
     hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT, getBinaryPort());
     hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT, getHttpPort());
     HiveMetaStore.HMSHandler.resetDefaultDBFlag();
+
+    Path scratchDir = new Path(baseDfsDir, "scratch");
+    fs.mkdirs(scratchDir);
+    System.setProperty(HiveConf.ConfVars.SCRATCHDIR.varname, scratchDir.toString());
+    System.setProperty(HiveConf.ConfVars.LOCALSCRATCHDIR.varname,
+        baseDir.getPath() + File.separator + "scratch");
   }
 
   public void start(Map<String, String> confOverlay) throws Exception {
@@ -78,6 +117,18 @@ public class MiniHS2 extends AbstractHiv
     verifyStarted();
     hiveServer2.stop();
     setStarted(false);
+    try {
+      if (mr != null) {
+        mr.shutdown();
+        mr = null;
+      }
+      if (dfs != null) {
+        dfs.shutdown();
+        dfs = null;
+      }
+    } catch (IOException e) {
+      // Ignore errors cleaning up miniMR
+    }
     FileUtils.deleteQuietly(baseDir);
   }
 
@@ -98,13 +149,37 @@ public class MiniHS2 extends AbstractHiv
     throw new IllegalStateException("HiveServer2 not running Thrift service");
   }
 
+  /**
+   * return connection URL for this server instance
+   * @return
+   */
   public String getJdbcURL() {
+    return getJdbcURL("default");
+  }
+
+  /**
+   * return connection URL for this server instance
+   * @param dbName - DB name to be included in the URL
+   * @return
+   */
+  public String getJdbcURL(String dbName) {
+    return getJdbcURL(dbName, "");
+  }
+
+  /**
+   * return connection URL for this server instance
+   * @param dbName - DB name to be included in the URL
+   * @param urlExtension - Addional string to be appended to URL
+   * @return
+   */
+  public String getJdbcURL(String dbName, String urlExtension) {
+    assertNotNull("URL extension shouldn't be null", urlExtension);
     String transportMode = getConfProperty(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname);
     if(transportMode != null && (transportMode.equalsIgnoreCase(HS2_HTTP_MODE))) {
-      return "jdbc:hive2://" + getHost() + ":" + getHttpPort() + "/default";
+      return "jdbc:hive2://" + getHost() + ":" + getHttpPort() + "/" + dbName;
     }
     else {
-      return "jdbc:hive2://" + getHost() + ":" + getBinaryPort() + "/default";
+      return "jdbc:hive2://" + getHost() + ":" + getBinaryPort() + "/" + dbName + urlExtension;
     }
   }
 

Modified: hive/branches/branch-0.13/pom.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/pom.xml?rev=1576584&r1=1576583&r2=1576584&view=diff
==============================================================================
--- hive/branches/branch-0.13/pom.xml (original)
+++ hive/branches/branch-0.13/pom.xml Wed Mar 12 05:10:30 2014
@@ -260,6 +260,11 @@
         <version>${wadl-resourcedoc-doclet.version}</version>
       </dependency>
       <dependency>
+        <groupId>com.sun.jersey</groupId>
+        <artifactId>jersey-servlet</artifactId>
+        <version>${jersey.version}</version>
+      </dependency>
+      <dependency>
         <groupId>commons-cli</groupId>
         <artifactId>commons-cli</artifactId>
         <version>${commons-cli.version}</version>
@@ -976,6 +981,11 @@
             <artifactId>hbase-server</artifactId>
             <version>${hbase.hadoop2.version}</version>
           </dependency>
+          <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-minicluster</artifactId>
+            <version>${hadoop-23.version}</version>
+          </dependency>
         </dependencies>
       </dependencyManagement>
     </profile>

Modified: hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/PlainSaslHelper.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/PlainSaslHelper.java?rev=1576584&r1=1576583&r2=1576584&view=diff
==============================================================================
--- hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/PlainSaslHelper.java (original)
+++ hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/PlainSaslHelper.java Wed Mar 12 05:10:30 2014
@@ -25,12 +25,14 @@ import javax.security.auth.callback.Call
 import javax.security.auth.callback.NameCallback;
 import javax.security.auth.callback.PasswordCallback;
 import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.AuthenticationException;
 import javax.security.sasl.AuthorizeCallback;
 import javax.security.sasl.SaslException;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hive.service.auth.PlainSaslServer.ExternalAuthenticationCallback;
 import org.apache.hive.service.auth.PlainSaslServer.SaslPlainProvider;
+import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods;
 import org.apache.hive.service.cli.thrift.TCLIService;
 import org.apache.hive.service.cli.thrift.TCLIService.Iface;
 import org.apache.hive.service.cli.thrift.ThriftCLIService;
@@ -44,24 +46,35 @@ import org.apache.thrift.transport.TTran
 public class PlainSaslHelper {
 
   private static class PlainServerCallbackHandler implements CallbackHandler {
+    private final AuthMethods authMethod;
+    public PlainServerCallbackHandler(String authMethodStr) throws AuthenticationException {
+      authMethod = AuthMethods.getValidAuthMethod(authMethodStr);
+    }
 
     @Override
     public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
-      ExternalAuthenticationCallback ac = null;
+      String userName = null;
+      String passWord = null;
+      AuthorizeCallback ac = null;
+
       for (int i = 0; i < callbacks.length; i++) {
-        if (callbacks[i] instanceof ExternalAuthenticationCallback) {
-          ac = (ExternalAuthenticationCallback) callbacks[i];
-          break;
+        if (callbacks[i] instanceof NameCallback) {
+          NameCallback nc = (NameCallback)callbacks[i];
+          userName = nc.getName();
+        } else if (callbacks[i] instanceof PasswordCallback) {
+          PasswordCallback pc = (PasswordCallback)callbacks[i];
+          passWord = new String(pc.getPassword());
+        } else if (callbacks[i] instanceof AuthorizeCallback) {
+          ac = (AuthorizeCallback) callbacks[i];
         } else {
           throw new UnsupportedCallbackException(callbacks[i]);
         }
       }
-
+      PasswdAuthenticationProvider provider =
+            AuthenticationProviderFactory.getAuthenticationProvider(authMethod);
+      provider.Authenticate(userName, passWord);
       if (ac != null) {
-        PasswdAuthenticationProvider provider =
-            AuthenticationProviderFactory.getAuthenticationProvider(ac.getAuthMethod());
-        provider.Authenticate(ac.getUserName(), ac.getPasswd());
-        ac.setAuthenticated(true);
+        ac.setAuthorized(true);
       }
     }
   }
@@ -123,11 +136,16 @@ public class PlainSaslHelper {
     java.security.Security.addProvider(new SaslPlainProvider());
   }
 
-  public static TTransportFactory getPlainTransportFactory(String authTypeStr) {
+  public static TTransportFactory getPlainTransportFactory(String authTypeStr)
+      throws LoginException {
     TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory();
-    saslFactory.addServerDefinition("PLAIN",
-        authTypeStr, null, new HashMap<String, String>(),
-        new PlainServerCallbackHandler());
+    try {
+      saslFactory.addServerDefinition("PLAIN",
+          authTypeStr, null, new HashMap<String, String>(),
+          new PlainServerCallbackHandler(authTypeStr));
+    } catch (AuthenticationException e) {
+      throw new LoginException ("Error setting callback handler" + e);
+    }
     return saslFactory;
   }
 

Modified: hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/PlainSaslServer.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/PlainSaslServer.java?rev=1576584&r1=1576583&r2=1576584&view=diff
==============================================================================
--- hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/PlainSaslServer.java (original)
+++ hive/branches/branch-0.13/service/src/java/org/apache/hive/service/auth/PlainSaslServer.java Wed Mar 12 05:10:30 2014
@@ -24,7 +24,10 @@ import java.util.Map;
 
 import javax.security.auth.callback.Callback;
 import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
 import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
 import javax.security.sasl.SaslException;
 import javax.security.sasl.SaslServer;
 import javax.security.sasl.SaslServerFactory;
@@ -41,45 +44,9 @@ public class PlainSaslServer implements 
   private final AuthMethods authMethod;
   private String user;
   private String passwd;
+  private String authzId;
   private final CallbackHandler handler;
 
-  // Callback for external authentication
-  // The authMethod indicates the type of authentication (LDAP, Unix, Windows)
-  public static class ExternalAuthenticationCallback implements Callback {
-    private final AuthMethods authMethod;
-    private final String userName;
-    private final String passwd;
-    private boolean authenticated;
-
-    public ExternalAuthenticationCallback(AuthMethods authMethod, String userName, String passwd) {
-      this.authMethod = authMethod;
-      this.userName = userName;
-      this.passwd = passwd;
-      authenticated = false;
-    }
-
-    public AuthMethods getAuthMethod() {
-      return authMethod;
-    }
-
-    public String getUserName() {
-      return userName;
-    }
-
-    public String getPasswd() {
-      return passwd;
-    }
-
-    public void setAuthenticated (boolean authenticated) {
-      this.authenticated = authenticated;
-    }
-
-    public boolean isAuthenticated () {
-      return authenticated;
-    }
-  }
-
-
   PlainSaslServer(CallbackHandler handler, String authMethodStr) throws SaslException {
     this.handler = handler;
     this.authMethod = AuthMethods.getValidAuthMethod(authMethodStr);
@@ -112,6 +79,12 @@ public class PlainSaslServer implements 
       }
       passwd = tokenList.removeLast();
       user = tokenList.removeLast();
+      // optional authzid
+      if (!tokenList.isEmpty()) {
+        authzId = tokenList.removeLast();
+      } else {
+        authzId = user;
+      }
       if (user == null || user.isEmpty()) {
         throw new SaslException("No user name provide");
       }
@@ -119,13 +92,15 @@ public class PlainSaslServer implements 
         throw new SaslException("No password name provide");
       }
 
-      // pass the user and passwd via AuthorizeCallback
-      // the caller needs to authenticate
-      ExternalAuthenticationCallback exAuth = new
-          ExternalAuthenticationCallback(authMethod, user, passwd);
-      Callback[] cbList = new Callback[] {exAuth};
+      NameCallback nameCallback = new NameCallback("User");
+      nameCallback.setName(user);
+      PasswordCallback pcCallback = new PasswordCallback("Password", false);
+      pcCallback.setPassword(passwd.toCharArray());
+      AuthorizeCallback acCallback = new AuthorizeCallback(user, authzId);
+
+      Callback[] cbList = new Callback[] {nameCallback, pcCallback, acCallback};
       handler.handle(cbList);
-      if (!exAuth.isAuthenticated()) {
+      if (!acCallback.isAuthorized()) {
         throw new SaslException("Authentication failed");
       }
     } catch (IllegalStateException eL) {