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 2017/05/15 06:52:36 UTC

hive git commit: HIVE-16416 : Service: move constants out from HiveAuthFactory (Zoltan Haindrich, reviewed by Peter Vary, Thejas Nair)

Repository: hive
Updated Branches:
  refs/heads/master c90aa83fb -> 72604208e


HIVE-16416 : Service: move constants out from HiveAuthFactory (Zoltan Haindrich, reviewed by Peter Vary, Thejas Nair)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/72604208
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/72604208
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/72604208

Branch: refs/heads/master
Commit: 72604208e9460483c1a05033cb37155aa8c2304d
Parents: c90aa83
Author: Zoltan Haindrich <ki...@rxd.hu>
Authored: Sun May 14 23:52:32 2017 -0700
Committer: Thejas M Nair <th...@hortonworks.com>
Committed: Sun May 14 23:52:32 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hive/beeline/ProxyAuthTest.java  |  4 +-
 .../hive/minikdc/TestHiveAuthFactory.java       |  5 +-
 .../hive/minikdc/TestJdbcWithMiniKdc.java       |  4 +-
 .../thrift/TestThriftCLIServiceWithBinary.java  |  5 +-
 .../thrift/TestThriftCLIServiceWithHttp.java    |  5 +-
 .../TestThriftHttpCLIServiceFeatures.java       |  5 +-
 .../thrift/ThriftCliServiceMessageSizeTest.java |  5 +-
 .../org/apache/hive/jdbc/HiveConnection.java    | 10 ++--
 .../hive/service/auth/HiveAuthConstants.java    | 43 ++++++++++++++++
 .../hive/service/auth/HiveAuthFactory.java      | 52 ++++++--------------
 .../service/cli/thrift/ThriftCLIService.java    |  7 +--
 .../service/cli/thrift/ThriftHttpServlet.java   |  7 +--
 .../cli/TestRetryingThriftCLIServiceClient.java |  4 +-
 .../service/cli/session/TestSessionHooks.java   |  4 +-
 .../thrift/ThriftCliServiceTestWithCookie.java  |  5 +-
 15 files changed, 96 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/beeline/src/test/org/apache/hive/beeline/ProxyAuthTest.java
----------------------------------------------------------------------
diff --git a/beeline/src/test/org/apache/hive/beeline/ProxyAuthTest.java b/beeline/src/test/org/apache/hive/beeline/ProxyAuthTest.java
index 0a08389..318857d 100644
--- a/beeline/src/test/org/apache/hive/beeline/ProxyAuthTest.java
+++ b/beeline/src/test/org/apache/hive/beeline/ProxyAuthTest.java
@@ -29,10 +29,10 @@ import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hive.jdbc.HiveConnection;
+import org.apache.hive.service.auth.HiveAuthConstants;
 import org.apache.hive.beeline.BeeLine;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.hive.shims.Utils;
-import org.apache.hive.service.auth.HiveAuthFactory;
 
 /**
  * Simple client application to test various direct and proxy connection to HiveServer2
@@ -203,7 +203,7 @@ public class ProxyAuthTest {
 
   private static void storeTokenInJobConf(String tokenStr) throws Exception {
     Utils.setTokenStr(Utils.getUGI(),
-          tokenStr, HiveAuthFactory.HS2_CLIENT_TOKEN);
+          tokenStr, HiveAuthConstants.HS2_CLIENT_TOKEN);
     System.out.println("Stored token " + tokenStr);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestHiveAuthFactory.java
----------------------------------------------------------------------
diff --git a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestHiveAuthFactory.java b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestHiveAuthFactory.java
index 0405ff3..e3a5190 100644
--- a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestHiveAuthFactory.java
+++ b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestHiveAuthFactory.java
@@ -20,6 +20,7 @@ package org.apache.hive.minikdc;
 import org.junit.Assert;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.service.auth.HiveAuthConstants;
 import org.apache.hive.service.auth.HiveAuthFactory;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -46,7 +47,7 @@ public class TestHiveAuthFactory {
    */
   @Test
   public void testStartTokenManagerForMemoryTokenStore() throws Exception {
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, HiveAuthFactory.AuthTypes.KERBEROS.getAuthName());
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, HiveAuthConstants.AuthTypes.KERBEROS.getAuthName());
     String principalName = miniHiveKdc.getFullHiveServicePrincipal();
     System.out.println("Principal: " + principalName);
 
@@ -68,7 +69,7 @@ public class TestHiveAuthFactory {
    */
   @Test
   public void testStartTokenManagerForDBTokenStore() throws Exception {
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, HiveAuthFactory.AuthTypes.KERBEROS.getAuthName());
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, HiveAuthConstants.AuthTypes.KERBEROS.getAuthName());
     String principalName = miniHiveKdc.getFullHiveServicePrincipal();
     System.out.println("Principal: " + principalName);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithMiniKdc.java
----------------------------------------------------------------------
diff --git a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithMiniKdc.java b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithMiniKdc.java
index daf0f7e..256262d 100644
--- a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithMiniKdc.java
+++ b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithMiniKdc.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hive.jdbc.HiveConnection;
 import org.apache.hive.jdbc.miniHS2.MiniHS2;
-import org.apache.hive.service.auth.HiveAuthFactory;
+import org.apache.hive.service.auth.HiveAuthConstants;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.session.HiveSessionHook;
 import org.apache.hive.service.cli.session.HiveSessionHookContext;
@@ -255,7 +255,7 @@ public class TestJdbcWithMiniKdc {
   protected void storeToken(String tokenStr, UserGroupInformation ugi)
       throws Exception {
     Utils.setTokenStr(ugi,
-        tokenStr, HiveAuthFactory.HS2_CLIENT_TOKEN);
+        tokenStr, HiveAuthConstants.HS2_CLIENT_TOKEN);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftCLIServiceWithBinary.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftCLIServiceWithBinary.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftCLIServiceWithBinary.java
index ceaf0e92..842c30f 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftCLIServiceWithBinary.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftCLIServiceWithBinary.java
@@ -21,7 +21,8 @@ package org.apache.hive.service.cli.thrift;
 import static org.junit.Assert.assertNotNull;
 
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hive.service.auth.HiveAuthFactory.AuthTypes;
+import org.apache.hive.service.auth.HiveAuthConstants;
+import org.apache.hive.service.auth.HiveAuthConstants.AuthTypes;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -54,7 +55,7 @@ public class TestThriftCLIServiceWithBinary extends ThriftCLIServiceTest {
     hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
     hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, host);
     hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT, port);
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, AuthTypes.NONE.toString());
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, HiveAuthConstants.AuthTypes.NONE.toString());
     hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, transportMode);
 
     startHiveServer2WithConf(hiveConf);

http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftCLIServiceWithHttp.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftCLIServiceWithHttp.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftCLIServiceWithHttp.java
index b1c6b67..3564272 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftCLIServiceWithHttp.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftCLIServiceWithHttp.java
@@ -29,8 +29,9 @@ import java.util.Map;
 
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hive.jdbc.HttpBasicAuthInterceptor;
+import org.apache.hive.service.auth.HiveAuthConstants;
+import org.apache.hive.service.auth.HiveAuthConstants.AuthTypes;
 import org.apache.hive.service.auth.HiveAuthFactory;
-import org.apache.hive.service.auth.HiveAuthFactory.AuthTypes;
 import org.apache.hive.service.rpc.thrift.TCLIService;
 import org.apache.hive.service.rpc.thrift.TOpenSessionReq;
 import org.apache.http.Header;
@@ -79,7 +80,7 @@ public class TestThriftCLIServiceWithHttp extends ThriftCLIServiceTest {
     hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
     hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, host);
     hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT, port);
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, AuthTypes.NOSASL.toString());
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, HiveAuthConstants.AuthTypes.NOSASL.toString());
     hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, transportMode);
     hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH, thriftHttpPath);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIServiceFeatures.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIServiceFeatures.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIServiceFeatures.java
index 1581f9b..1911d2c 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIServiceFeatures.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIServiceFeatures.java
@@ -44,7 +44,8 @@ import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext;
 import org.apache.hive.jdbc.HttpBasicAuthInterceptor;
-import org.apache.hive.service.auth.HiveAuthFactory.AuthTypes;
+import org.apache.hive.service.auth.HiveAuthConstants;
+import org.apache.hive.service.auth.HiveAuthConstants.AuthTypes;
 import org.apache.hive.service.rpc.thrift.TCLIService;
 import org.apache.hive.service.rpc.thrift.TExecuteStatementReq;
 import org.apache.hive.service.rpc.thrift.TOpenSessionReq;
@@ -133,7 +134,7 @@ public class TestThriftHttpCLIServiceFeatures  {
     hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
     hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, ThriftCLIServiceTest.host);
     hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT, ThriftCLIServiceTest.port);
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, AuthTypes.NOSASL.toString());
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, HiveAuthConstants.AuthTypes.NOSASL.toString());
     hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, transportMode);
     hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH, thriftHttpPath);
     hiveConf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false);

http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/ThriftCliServiceMessageSizeTest.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/ThriftCliServiceMessageSizeTest.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/ThriftCliServiceMessageSizeTest.java
index 7148307..c859b8f 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/ThriftCliServiceMessageSizeTest.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/ThriftCliServiceMessageSizeTest.java
@@ -30,7 +30,8 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hive.service.Service;
-import org.apache.hive.service.auth.HiveAuthFactory.AuthTypes;
+import org.apache.hive.service.auth.HiveAuthConstants;
+import org.apache.hive.service.auth.HiveAuthConstants.AuthTypes;
 import org.apache.hive.service.cli.SessionHandle;
 import org.apache.hive.service.server.HiveServer2;
 import org.junit.After;
@@ -110,7 +111,7 @@ public class ThriftCliServiceMessageSizeTest {
     hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
     hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, host);
     hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT, port);
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, AuthTypes.NONE.toString());
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, HiveAuthConstants.AuthTypes.NONE.toString());
     hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, transportMode);
 
     HiveServer2 hiveServer2 = new HiveServer2();

http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java b/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
index fb18adb..8b37ab0 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
@@ -21,7 +21,7 @@ package org.apache.hive.jdbc;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.common.auth.HiveAuthUtils;
 import org.apache.hive.jdbc.Utils.JdbcConnectionParams;
-import org.apache.hive.service.auth.HiveAuthFactory;
+import org.apache.hive.service.auth.HiveAuthConstants;
 import org.apache.hive.service.auth.KerberosSaslHelper;
 import org.apache.hive.service.auth.PlainSaslHelper;
 import org.apache.hive.service.auth.SaslQOP;
@@ -636,7 +636,7 @@ public class HiveConnection implements java.sql.Connection {
     if (JdbcConnectionParams.AUTH_TOKEN.equalsIgnoreCase(jdbcConnConf.get(JdbcConnectionParams.AUTH_TYPE))) {
       // check delegation token in job conf if any
       try {
-        tokenStr = org.apache.hadoop.hive.shims.Utils.getTokenStrForm(HiveAuthFactory.HS2_CLIENT_TOKEN);
+        tokenStr = org.apache.hadoop.hive.shims.Utils.getTokenStrForm(HiveAuthConstants.HS2_CLIENT_TOKEN);
       } catch (IOException e) {
         throw new SQLException("Error reading token ", e);
       }
@@ -664,9 +664,9 @@ public class HiveConnection implements java.sql.Connection {
 
     // set the session configuration
     Map<String, String> sessVars = connParams.getSessionVars();
-    if (sessVars.containsKey(HiveAuthFactory.HS2_PROXY_USER)) {
-      openConf.put(HiveAuthFactory.HS2_PROXY_USER,
-          sessVars.get(HiveAuthFactory.HS2_PROXY_USER));
+    if (sessVars.containsKey(HiveAuthConstants.HS2_PROXY_USER)) {
+      openConf.put(HiveAuthConstants.HS2_PROXY_USER,
+          sessVars.get(HiveAuthConstants.HS2_PROXY_USER));
     }
     openReq.setConfiguration(openConf);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/service/src/java/org/apache/hive/service/auth/HiveAuthConstants.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/HiveAuthConstants.java b/service/src/java/org/apache/hive/service/auth/HiveAuthConstants.java
new file mode 100644
index 0000000..a49f2f1
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/HiveAuthConstants.java
@@ -0,0 +1,43 @@
+/**
+ * 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;
+
+public class HiveAuthConstants {
+  public enum AuthTypes {
+    NOSASL("NOSASL"),
+    NONE("NONE"),
+    LDAP("LDAP"),
+    KERBEROS("KERBEROS"),
+    CUSTOM("CUSTOM"),
+    PAM("PAM");
+
+    private final String authType;
+
+    AuthTypes(String authType) {
+      this.authType = authType;
+    }
+
+    public String getAuthName() {
+      return authType;
+    }
+  }
+
+  public static final String HS2_PROXY_USER = "hive.server2.proxy.user";
+  public static final String HS2_CLIENT_TOKEN = "hiveserver2ClientToken";
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java b/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
index dcb6338..541fe5e 100644
--- a/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
+++ b/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
@@ -56,27 +56,6 @@ import org.slf4j.LoggerFactory;
 public class HiveAuthFactory {
   private static final Logger LOG = LoggerFactory.getLogger(HiveAuthFactory.class);
 
-
-  public enum AuthTypes {
-    NOSASL("NOSASL"),
-    NONE("NONE"),
-    LDAP("LDAP"),
-    KERBEROS("KERBEROS"),
-    CUSTOM("CUSTOM"),
-    PAM("PAM");
-
-    private final String authType;
-
-    AuthTypes(String authType) {
-      this.authType = authType;
-    }
-
-    public String getAuthName() {
-      return authType;
-    }
-
-  }
-
   private HadoopThriftAuthBridge.Server saslServer;
   private String authTypeStr;
   private final String transportMode;
@@ -84,9 +63,6 @@ public class HiveAuthFactory {
   private String hadoopAuth;
   private HiveDelegationTokenManager delegationTokenManager = null;
 
-  public static final String HS2_PROXY_USER = "hive.server2.proxy.user";
-  public static final String HS2_CLIENT_TOKEN = "hiveserver2ClientToken";
-
   public HiveAuthFactory(HiveConf conf) throws TTransportException {
     this.conf = conf;
     transportMode = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE);
@@ -99,9 +75,9 @@ public class HiveAuthFactory {
     // In http mode we use NOSASL as the default auth type
     if (authTypeStr == null) {
       if ("http".equalsIgnoreCase(transportMode)) {
-        authTypeStr = AuthTypes.NOSASL.getAuthName();
+        authTypeStr = HiveAuthConstants.AuthTypes.NOSASL.getAuthName();
       } else {
-        authTypeStr = AuthTypes.NONE.getAuthName();
+        authTypeStr = HiveAuthConstants.AuthTypes.NONE.getAuthName();
       }
     }
     if (isSASLWithKerberizedHadoop()) {
@@ -156,12 +132,12 @@ public class HiveAuthFactory {
       } catch (TTransportException e) {
         throw new LoginException(e.getMessage());
       }
-      if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) {
+      if (authTypeStr.equalsIgnoreCase(HiveAuthConstants.AuthTypes.KERBEROS.getAuthName())) {
         // no-op
-      } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NONE.getAuthName()) ||
-          authTypeStr.equalsIgnoreCase(AuthTypes.LDAP.getAuthName()) ||
-          authTypeStr.equalsIgnoreCase(AuthTypes.PAM.getAuthName()) ||
-          authTypeStr.equalsIgnoreCase(AuthTypes.CUSTOM.getAuthName())) {
+      } else if (authTypeStr.equalsIgnoreCase(HiveAuthConstants.AuthTypes.NONE.getAuthName()) ||
+          authTypeStr.equalsIgnoreCase(HiveAuthConstants.AuthTypes.LDAP.getAuthName()) ||
+          authTypeStr.equalsIgnoreCase(HiveAuthConstants.AuthTypes.PAM.getAuthName()) ||
+          authTypeStr.equalsIgnoreCase(HiveAuthConstants.AuthTypes.CUSTOM.getAuthName())) {
         try {
           serverTransportFactory.addServerDefinition("PLAIN",
               authTypeStr, null, new HashMap<String, String>(),
@@ -173,12 +149,12 @@ public class HiveAuthFactory {
         throw new LoginException("Unsupported authentication type " + authTypeStr);
       }
       transportFactory = saslServer.wrapTransportFactory(serverTransportFactory);
-    } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NONE.getAuthName()) ||
-          authTypeStr.equalsIgnoreCase(AuthTypes.LDAP.getAuthName()) ||
-          authTypeStr.equalsIgnoreCase(AuthTypes.PAM.getAuthName()) ||
-          authTypeStr.equalsIgnoreCase(AuthTypes.CUSTOM.getAuthName())) {
+    } else if (authTypeStr.equalsIgnoreCase(HiveAuthConstants.AuthTypes.NONE.getAuthName()) ||
+          authTypeStr.equalsIgnoreCase(HiveAuthConstants.AuthTypes.LDAP.getAuthName()) ||
+          authTypeStr.equalsIgnoreCase(HiveAuthConstants.AuthTypes.PAM.getAuthName()) ||
+          authTypeStr.equalsIgnoreCase(HiveAuthConstants.AuthTypes.CUSTOM.getAuthName())) {
        transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr);
-    } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NOSASL.getAuthName())) {
+    } else if (authTypeStr.equalsIgnoreCase(HiveAuthConstants.AuthTypes.NOSASL.getAuthName())) {
       transportFactory = new TTransportFactory();
     } else {
       throw new LoginException("Unsupported authentication type " + authTypeStr);
@@ -218,7 +194,7 @@ public class HiveAuthFactory {
 
   public boolean isSASLWithKerberizedHadoop() {
     return "kerberos".equalsIgnoreCase(hadoopAuth)
-        && !authTypeStr.equalsIgnoreCase(AuthTypes.NOSASL.getAuthName());
+        && !authTypeStr.equalsIgnoreCase(HiveAuthConstants.AuthTypes.NOSASL.getAuthName());
   }
 
   public boolean isSASLKerberosUser() {
@@ -259,7 +235,7 @@ public class HiveAuthFactory {
 
     try {
       String tokenStr = delegationTokenManager.getDelegationTokenWithService(owner, renewer,
-          HS2_CLIENT_TOKEN, remoteAddr);
+          HiveAuthConstants.HS2_CLIENT_TOKEN, remoteAddr);
       if (tokenStr == null || tokenStr.isEmpty()) {
         throw new HiveSQLException(
             "Received empty retrieving delegation token for user " + owner, "08S01");

http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
index 0fdc8d9..2b252d2 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hive.service.AbstractService;
 import org.apache.hive.service.ServiceException;
 import org.apache.hive.service.ServiceUtils;
+import org.apache.hive.service.auth.HiveAuthConstants;
 import org.apache.hive.service.auth.HiveAuthFactory;
 import org.apache.hive.service.auth.TSetIpAddressProcessor;
 import org.apache.hive.service.cli.CLIService;
@@ -820,8 +821,8 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       LOG.debug("Proxy user from query string: " + proxyUser);
     }
 
-    if (proxyUser == null && sessionConf != null && sessionConf.containsKey(HiveAuthFactory.HS2_PROXY_USER)) {
-      String proxyUserFromThriftBody = sessionConf.get(HiveAuthFactory.HS2_PROXY_USER);
+    if (proxyUser == null && sessionConf != null && sessionConf.containsKey(HiveAuthConstants.HS2_PROXY_USER)) {
+      String proxyUserFromThriftBody = sessionConf.get(HiveAuthConstants.HS2_PROXY_USER);
       LOG.debug("Proxy user from thrift body: " + proxyUserFromThriftBody);
       proxyUser = proxyUserFromThriftBody;
     }
@@ -836,7 +837,7 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
     }
 
     // If there's no authentication, then directly substitute the user
-    if (HiveAuthFactory.AuthTypes.NONE.toString().
+    if (HiveAuthConstants.AuthTypes.NONE.toString().
         equalsIgnoreCase(hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION))) {
       return proxyUser;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
index fbe6da4..cda736c 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthentica
 import org.apache.hive.service.CookieSigner;
 import org.apache.hive.service.auth.AuthenticationProviderFactory;
 import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods;
+import org.apache.hive.service.auth.HiveAuthConstants;
 import org.apache.hive.service.auth.HiveAuthFactory;
 import org.apache.hive.service.auth.HttpAuthUtils;
 import org.apache.hive.service.auth.HttpAuthenticationException;
@@ -192,7 +193,7 @@ public class ThriftHttpServlet extends TServlet {
 
       // Generate new cookie and add it to the response
       if (requireNewCookie &&
-          !authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) {
+          !authType.equalsIgnoreCase(HiveAuthConstants.AuthTypes.NOSASL.toString())) {
         String cookieToken = HttpAuthUtils.createCookieToken(clientUserName);
         Cookie hs2Cookie = createCookie(signer.signCookie(cookieToken));
 
@@ -354,7 +355,7 @@ public class ThriftHttpServlet extends TServlet {
       throws HttpAuthenticationException {
     String userName = getUsername(request, authType);
     // No-op when authType is NOSASL
-    if (!authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) {
+    if (!authType.equalsIgnoreCase(HiveAuthConstants.AuthTypes.NOSASL.toString())) {
       try {
         AuthMethods authMethod = AuthMethods.getValidAuthMethod(authType);
         PasswdAuthenticationProvider provider =
@@ -567,7 +568,7 @@ public class ThriftHttpServlet extends TServlet {
   }
 
   private boolean isKerberosAuthMode(String authType) {
-    return authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString());
+    return authType.equalsIgnoreCase(HiveAuthConstants.AuthTypes.KERBEROS.toString());
   }
 
   private static String getDoAsQueryParam(String queryString) {

http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/service/src/test/org/apache/hive/service/cli/TestRetryingThriftCLIServiceClient.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/cli/TestRetryingThriftCLIServiceClient.java b/service/src/test/org/apache/hive/service/cli/TestRetryingThriftCLIServiceClient.java
index 79953c4..2c4a6fe 100644
--- a/service/src/test/org/apache/hive/service/cli/TestRetryingThriftCLIServiceClient.java
+++ b/service/src/test/org/apache/hive/service/cli/TestRetryingThriftCLIServiceClient.java
@@ -20,7 +20,7 @@ package org.apache.hive.service.cli;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hive.service.Service;
-import org.apache.hive.service.auth.HiveAuthFactory;
+import org.apache.hive.service.auth.HiveAuthConstants;
 import org.apache.hive.service.cli.session.HiveSession;
 import org.apache.hive.service.cli.thrift.RetryingThriftCLIServiceClient;
 import org.apache.hive.service.cli.thrift.ThriftCLIService;
@@ -54,7 +54,7 @@ public class TestRetryingThriftCLIServiceClient {
     hiveConf.setVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, "localhost");
     hiveConf.setIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT, 15000);
     hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
-    hiveConf.setVar(HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION, HiveAuthFactory.AuthTypes.NONE.toString());
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION, HiveAuthConstants.AuthTypes.NONE.toString());
     hiveConf.setVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE, "binary");
     hiveConf.setIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_CLIENT_RETRY_LIMIT, 3);
     hiveConf.setIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_CLIENT_CONNECTION_RETRY_LIMIT, 3);

http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java b/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java
index 2878825..ff1d837 100644
--- a/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java
+++ b/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java
@@ -28,7 +28,7 @@ import junit.framework.TestCase;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hive.service.auth.HiveAuthFactory;
+import org.apache.hive.service.auth.HiveAuthConstants;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.SessionHandle;
 import org.apache.hive.service.cli.thrift.EmbeddedThriftBinaryCLIService;
@@ -90,7 +90,7 @@ public class TestSessionHooks extends TestCase {
     String connectingUser = "user1";
     String proxyUser = System.getProperty("user.name");
     Map<String, String>sessConf = new HashMap<String,String>();
-    sessConf.put(HiveAuthFactory.HS2_PROXY_USER, proxyUser);
+    sessConf.put(HiveAuthConstants.HS2_PROXY_USER, proxyUser);
     sessionUserName = proxyUser;
     SessionHandle sessionHandle = client.openSession(connectingUser, "foobar", sessConf);
     Assert.assertEquals(1, SessionHookTest.runCount.get());

http://git-wip-us.apache.org/repos/asf/hive/blob/72604208/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java b/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java
index 6fec947..13f582d 100644
--- a/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java
+++ b/service/src/test/org/apache/hive/service/cli/thrift/ThriftCliServiceTestWithCookie.java
@@ -31,7 +31,8 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hive.service.Service;
-import org.apache.hive.service.auth.HiveAuthFactory.AuthTypes;
+import org.apache.hive.service.auth.HiveAuthConstants;
+import org.apache.hive.service.auth.HiveAuthConstants.AuthTypes;
 import org.apache.hive.service.cli.OperationHandle;
 import org.apache.hive.service.cli.OperationState;
 import org.apache.hive.service.cli.OperationStatus;
@@ -79,7 +80,7 @@ public class ThriftCliServiceTestWithCookie {
     hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
     hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, host);
     hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT, port);
-    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, AuthTypes.NOSASL.toString());
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, HiveAuthConstants.AuthTypes.NOSASL.toString());
 
     startHiveServer2WithConf(hiveConf);