You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2014/02/25 05:34:13 UTC

svn commit: r1571560 - in /hive/trunk: itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/ itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/ ql/src/java/org/apache/hadoop/hive/ql/security/ ql/src/java/org/apache/hadoop/hive/ql...

Author: thejas
Date: Tue Feb 25 04:34:13 2014
New Revision: 1571560

URL: http://svn.apache.org/r1571560
Log:
HIVE-6478 : SQL std auth - pass username from hiveserver2 to sessionstate (Thejas Nair, reviewed by Ashutosh Chauhan)

Added:
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestMiniHS2.java
Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/SessionStateUserAuthenticator.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestSessionUserName.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java

Added: hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java?rev=1571560&view=auto
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java (added)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java Tue Feb 25 04:34:13 2014
@@ -0,0 +1,115 @@
+/**
+ * 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.authorization;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.HashMap;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.security.SessionStateUserAuthenticator;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+import org.apache.hive.jdbc.miniHS2.MiniHS2;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test SQL standard authorization with jdbc/hiveserver2
+ */
+public class TestJdbcWithSQLAuthorization {
+  private static MiniHS2 miniHS2 = null;
+
+  @BeforeClass
+  public static void beforeTest() throws Exception {
+    Class.forName(MiniHS2.getJdbcDriverName());
+    HiveConf conf = new HiveConf();
+    conf.setVar(ConfVars.HIVE_AUTHORIZATION_MANAGER, SQLStdHiveAuthorizerFactory.class.getName());
+    conf.setVar(ConfVars.HIVE_AUTHENTICATOR_MANAGER, SessionStateUserAuthenticator.class.getName());
+    conf.setBoolVar(ConfVars.HIVE_AUTHORIZATION_ENABLED, true);
+    conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    conf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
+
+    miniHS2 = new MiniHS2(conf);
+    miniHS2.start(new HashMap<String, String>());
+  }
+
+  @AfterClass
+  public static void afterTest() throws Exception {
+    if (miniHS2.isStarted()) {
+      miniHS2.stop();
+    }
+  }
+
+  @Test
+  public void testAuthorization1() throws Exception {
+
+    String tableName1 = "test_jdbc_sql_auth1";
+    String tableName2 = "test_jdbc_sql_auth2";
+    // using different code blocks so that jdbc variables are not accidently re-used
+    // between the actions. Different connection/statement object should be used for each action.
+    {
+      // create tables as user1
+      Connection hs2Conn = getConnection("user1");
+
+      Statement stmt = hs2Conn.createStatement();
+
+      // create tables
+      stmt.execute("create table " + tableName1 + "(i int) ");
+      stmt.execute("create table " + tableName2 + "(i int) ");
+      stmt.close();
+      hs2Conn.close();
+    }
+    {
+      // try dropping table as user1 - should succeed
+      Connection hs2Conn = getConnection("user1");
+      Statement stmt = hs2Conn.createStatement();
+      stmt.execute("drop table " + tableName1);
+    }
+
+    {
+      // try dropping table as user2 - should fail
+      Connection hs2Conn = getConnection("user2");
+      try {
+        Statement stmt = hs2Conn.createStatement();
+        stmt.execute("drop table " + tableName2);
+        fail("Exception due to authorization failure is expected");
+      } catch (SQLException e) {
+        String msg = e.getMessage();
+        System.err.println("Got SQLException with message " + msg);
+        // check parts of the error, not the whole string so as not to tightly
+        // couple the error message with test
+        assertTrue("Checking permission denied error", msg.contains("user2"));
+        assertTrue("Checking permission denied error", msg.contains(tableName2));
+        assertTrue("Checking permission denied error", msg.contains("OBJECT OWNERSHIP"));
+      }
+    }
+  }
+
+  private Connection getConnection(String userName) throws SQLException {
+    return DriverManager.getConnection(miniHS2.getJdbcURL(), userName, "bar");
+  }
+
+}

Added: hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestMiniHS2.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestMiniHS2.java?rev=1571560&view=auto
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestMiniHS2.java (added)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestMiniHS2.java Tue Feb 25 04:34:13 2014
@@ -0,0 +1,96 @@
+/**
+ * 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.miniHS2;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.HashMap;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestMiniHS2 {
+
+  private MiniHS2 miniHS2;
+
+  @BeforeClass
+  public static void beforeTest() throws Exception {
+    Class.forName(MiniHS2.getJdbcDriverName());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    miniHS2.stop();
+  }
+
+  /**
+   * Test if the MiniHS2 configuration gets passed down to the session
+   * configuration
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testConfInSession() throws Exception {
+    HiveConf hiveConf = new HiveConf();
+    final String DUMMY_CONF_KEY = "hive.test.minihs2.dummy.config";
+    final String DUMMY_CONF_VAL = "dummy.val";
+    hiveConf.set(DUMMY_CONF_KEY, DUMMY_CONF_VAL);
+
+    // also check a config that has default in hiveconf
+    final String STATS_TIMEOUT_KEY = ConfVars.HIVE_STATS_JDBC_TIMEOUT.varname;
+    final String STATS_TIMEOUT = "2562";
+    hiveConf.set(STATS_TIMEOUT_KEY, STATS_TIMEOUT);
+
+    // check the config used very often!
+    hiveConf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+
+    miniHS2 = new MiniHS2(hiveConf);
+    miniHS2.start(new HashMap<String, String>());
+
+    Connection hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL(),
+        System.getProperty("user.name"), "bar");
+    Statement stmt = hs2Conn.createStatement();
+
+    checkConfVal(DUMMY_CONF_KEY, DUMMY_CONF_KEY + "=" + DUMMY_CONF_VAL, stmt);
+    checkConfVal(STATS_TIMEOUT_KEY, STATS_TIMEOUT_KEY + "=" + STATS_TIMEOUT, stmt);
+    checkConfVal(ConfVars.HIVE_SUPPORT_CONCURRENCY.varname,
+        ConfVars.HIVE_SUPPORT_CONCURRENCY.varname + "=" + "false", stmt);
+
+    stmt.close();
+    hs2Conn.close();
+
+  }
+
+  private void checkConfVal(String confKey, String confResult, Statement stmt) throws SQLException {
+    ResultSet res = stmt.executeQuery("set " + confKey);
+    assertTrue(res.next());
+    assertEquals("Expected config result", confResult, res.getString(1));
+    res.close();
+  }
+
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/SessionStateUserAuthenticator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/SessionStateUserAuthenticator.java?rev=1571560&r1=1571559&r2=1571560&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/SessionStateUserAuthenticator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/SessionStateUserAuthenticator.java Tue Feb 25 04:34:13 2014
@@ -36,10 +36,6 @@ public class SessionStateUserAuthenticat
   protected Configuration conf;
   private SessionState sessionState;
 
-  public SessionStateUserAuthenticator(SessionState sessionState){
-    this.sessionState = sessionState;
-  }
-
   @Override
   public List<String> getGroupNames() {
     return groupNames;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java?rev=1571560&r1=1571559&r2=1571560&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java Tue Feb 25 04:34:13 2014
@@ -447,10 +447,10 @@ public class SQLStdHiveAccessController 
     try {
       roles = getCurrentRoles();
     } catch (Exception e) {
-        throw new HiveAuthzPluginException(e);
+      throw new HiveAuthzPluginException(e);
     }
-    for (HiveRole role : roles){
-    if (role.getRoleName().equalsIgnoreCase(HiveMetaStore.ADMIN)) {
+    for (HiveRole role : roles) {
+      if (role.getRoleName().equalsIgnoreCase(HiveMetaStore.ADMIN)) {
         return true;
       }
     }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java?rev=1571560&r1=1571559&r2=1571560&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java Tue Feb 25 04:34:13 2014
@@ -54,7 +54,6 @@ import org.apache.hadoop.hive.ql.metadat
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
-import org.apache.hadoop.hive.ql.security.SessionStateUserAuthenticator;
 import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.DisallowTransformHook;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizer;
@@ -350,12 +349,6 @@ public class SessionState {
     try {
       authenticator = HiveUtils.getAuthenticator(getConf(),
           HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER);
-
-      if (userName != null) {
-        // if username is set through the session, use an authenticator that
-        // just returns the sessionstate user
-        authenticator = new SessionStateUserAuthenticator(this);
-      }
       authenticator.setSessionState(this);
 
       authorizer = HiveUtils.getAuthorizeProviderManager(getConf(),

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestSessionUserName.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestSessionUserName.java?rev=1571560&r1=1571559&r2=1571560&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestSessionUserName.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestSessionUserName.java Tue Feb 25 04:34:13 2014
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
+import org.apache.hadoop.hive.ql.security.SessionStateUserAuthenticator;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizer;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizerFactory;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizerImpl;
@@ -96,6 +97,8 @@ public class TestSessionUserName {
     HiveConf conf = new HiveConf();
     conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
         HiveAuthorizerStoringUserNameFactory.class.getName());
+    conf.setVar(HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER,
+        SessionStateUserAuthenticator.class.getName());
     return conf;
   }
 

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java?rev=1571560&r1=1571559&r2=1571560&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java Tue Feb 25 04:34:13 2014
@@ -19,7 +19,6 @@
 package org.apache.hive.service.cli.session;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -66,8 +65,7 @@ public class HiveSessionImpl implements 
 
   private String username;
   private final String password;
-  private final Map<String, String> sessionConf = new HashMap<String, String>();
-  private final HiveConf hiveConf = new HiveConf();
+  private final HiveConf hiveConf;
   private final SessionState sessionState;
 
   private static final String FETCH_WORK_SERDE_CLASS =
@@ -81,13 +79,14 @@ public class HiveSessionImpl implements 
   private final Set<OperationHandle> opHandleSet = new HashSet<OperationHandle>();
 
   public HiveSessionImpl(TProtocolVersion protocol, String username, String password,
-      Map<String, String> sessionConf) {
+      HiveConf serverhiveConf, Map<String, String> sessionConfMap) {
     this.username = username;
     this.password = password;
     this.sessionHandle = new SessionHandle(protocol);
+    this.hiveConf = new HiveConf(serverhiveConf);
 
-    if (sessionConf != null) {
-      for (Map.Entry<String, String> entry : sessionConf.entrySet()) {
+    if (sessionConfMap != null) {
+      for (Map.Entry<String, String> entry : sessionConfMap.entrySet()) {
         hiveConf.set(entry.getKey(), entry.getValue());
       }
     }
@@ -98,18 +97,21 @@ public class HiveSessionImpl implements 
     hiveConf.set(ListSinkOperator.OUTPUT_FORMATTER,
         FetchFormatter.ThriftFormatter.class.getName());
     hiveConf.setInt(ListSinkOperator.OUTPUT_PROTOCOL, protocol.getValue());
-    sessionState = new SessionState(hiveConf);
+    sessionState = new SessionState(hiveConf, username);
     SessionState.start(sessionState);
   }
 
+  @Override
   public TProtocolVersion getProtocolVersion() {
     return sessionHandle.getProtocolVersion();
   }
 
+  @Override
   public SessionManager getSessionManager() {
     return sessionManager;
   }
 
+  @Override
   public void setSessionManager(SessionManager sessionManager) {
     this.sessionManager = sessionManager;
   }
@@ -118,6 +120,7 @@ public class HiveSessionImpl implements 
     return operationManager;
   }
 
+  @Override
   public void setOperationManager(OperationManager operationManager) {
     this.operationManager = operationManager;
   }
@@ -133,23 +136,28 @@ public class HiveSessionImpl implements 
     // no need to release sessionState...
   }
 
+  @Override
   public SessionHandle getSessionHandle() {
     return sessionHandle;
   }
 
+  @Override
   public String getUsername() {
     return username;
   }
 
+  @Override
   public String getPassword() {
     return password;
   }
 
+  @Override
   public HiveConf getHiveConf() {
     hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHOUTPUTSERDE, FETCH_WORK_SERDE_CLASS);
     return hiveConf;
   }
 
+  @Override
   public IMetaStoreClient getMetaStoreClient() throws HiveSQLException {
     if (metastoreClient == null) {
       try {
@@ -161,6 +169,7 @@ public class HiveSessionImpl implements 
     return metastoreClient;
   }
 
+  @Override
   public GetInfoValue getInfo(GetInfoType getInfoType)
       throws HiveSQLException {
     acquire();
@@ -187,11 +196,13 @@ public class HiveSessionImpl implements 
     }
   }
 
+  @Override
   public OperationHandle executeStatement(String statement, Map<String, String> confOverlay)
       throws HiveSQLException {
     return executeStatementInternal(statement, confOverlay, false);
   }
 
+  @Override
   public OperationHandle executeStatementAsync(String statement, Map<String, String> confOverlay)
       throws HiveSQLException {
     return executeStatementInternal(statement, confOverlay, true);
@@ -222,6 +233,7 @@ public class HiveSessionImpl implements 
     }
   }
 
+  @Override
   public OperationHandle getTypeInfo()
       throws HiveSQLException {
     acquire();
@@ -241,6 +253,7 @@ public class HiveSessionImpl implements 
     }
   }
 
+  @Override
   public OperationHandle getCatalogs()
       throws HiveSQLException {
     acquire();
@@ -260,6 +273,7 @@ public class HiveSessionImpl implements 
     }
   }
 
+  @Override
   public OperationHandle getSchemas(String catalogName, String schemaName)
       throws HiveSQLException {
     acquire();
@@ -280,6 +294,7 @@ public class HiveSessionImpl implements 
     }
   }
 
+  @Override
   public OperationHandle getTables(String catalogName, String schemaName, String tableName,
       List<String> tableTypes)
       throws HiveSQLException {
@@ -301,6 +316,7 @@ public class HiveSessionImpl implements 
     }
   }
 
+  @Override
   public OperationHandle getTableTypes()
       throws HiveSQLException {
     acquire();
@@ -320,6 +336,7 @@ public class HiveSessionImpl implements 
     }
   }
 
+  @Override
   public OperationHandle getColumns(String catalogName, String schemaName,
       String tableName, String columnName)  throws HiveSQLException {
     acquire();
@@ -340,6 +357,7 @@ public class HiveSessionImpl implements 
     }
   }
 
+  @Override
   public OperationHandle getFunctions(String catalogName, String schemaName, String functionName)
       throws HiveSQLException {
     acquire();
@@ -360,6 +378,7 @@ public class HiveSessionImpl implements 
     }
   }
 
+  @Override
   public void close() throws HiveSQLException {
     try {
       acquire();
@@ -388,13 +407,16 @@ public class HiveSessionImpl implements 
     }
   }
 
+  @Override
   public SessionState getSessionState() {
     return sessionState;
   }
 
+  @Override
   public String getUserName() {
     return username;
   }
+  @Override
   public void setUserName(String userName) {
     this.username = userName;
   }

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java?rev=1571560&r1=1571559&r2=1571560&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java Tue Feb 25 04:34:13 2014
@@ -21,6 +21,7 @@ package org.apache.hive.service.cli.sess
 import java.io.IOException;
 import java.util.Map;
 
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.shims.ShimLoader;
@@ -42,8 +43,8 @@ public class HiveSessionImplwithUGI exte
   private HiveSession proxySession = null;
 
   public HiveSessionImplwithUGI(TProtocolVersion protocol, String username, String password,
-      Map<String, String> sessionConf, String delegationToken) throws HiveSQLException {
-    super(protocol, username, password, sessionConf);
+      HiveConf hiveConf, Map<String, String> sessionConf, String delegationToken) throws HiveSQLException {
+    super(protocol, username, password, hiveConf, sessionConf);
     setSessionUGI(username);
     setDelegationToken(delegationToken);
   }

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java?rev=1571560&r1=1571559&r2=1571560&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java Tue Feb 25 04:34:13 2014
@@ -107,11 +107,11 @@ public class SessionManager extends Comp
     HiveSession session;
     if (withImpersonation) {
       HiveSessionImplwithUGI hiveSessionUgi = new HiveSessionImplwithUGI(protocol, username, password,
-        sessionConf, delegationToken);
+        hiveConf, sessionConf, delegationToken);
       session = HiveSessionProxy.getProxy(hiveSessionUgi, hiveSessionUgi.getSessionUgi());
       hiveSessionUgi.setProxySession(session);
     } else {
-      session = new HiveSessionImpl(protocol, username, password, sessionConf);
+      session = new HiveSessionImpl(protocol, username, password, hiveConf, sessionConf);
     }
     session.setSessionManager(this);
     session.setOperationManager(operationManager);