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/09/19 23:53:04 UTC

svn commit: r1626347 - in /hive/trunk: itests/util/src/main/java/org/apache/hadoop/hive/ql/security/ itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/ ql/src/java/org/apache/hadoop/hive/ql/processors/ ql/src/java...

Author: thejas
Date: Fri Sep 19 21:53:03 2014
New Revision: 1626347

URL: http://svn.apache.org/r1626347
Log:
HIVE-8045 : SQL standard auth with cli - Errors and configuration issues (Thejas Nair, reviewed by Jason Dere)

Added:
    hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/MetastoreAuthzAPIDisallowAuthorizer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/DummyHiveAuthorizationValidator.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdConfOnlyAuthorizerFactory.java
    hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_createtab_noauthzapi.q
    hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_nonsql.q
    hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_stdconfigauth.q
    hive/trunk/ql/src/test/results/clientpositive/authorization_cli_createtab_noauthzapi.q.out
    hive/trunk/ql/src/test/results/clientpositive/authorization_cli_nonsql.q.out
    hive/trunk/ql/src/test/results/clientpositive/authorization_cli_stdconfigauth.q.out
Modified:
    hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidatorForTest.java
    hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactoryForTest.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandUtil.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLAuthorizationUtils.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/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java
    hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_createtab.q

Added: hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/MetastoreAuthzAPIDisallowAuthorizer.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/MetastoreAuthzAPIDisallowAuthorizer.java?rev=1626347&view=auto
==============================================================================
--- hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/MetastoreAuthzAPIDisallowAuthorizer.java (added)
+++ hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/MetastoreAuthzAPIDisallowAuthorizer.java Fri Sep 19 21:53:03 2014
@@ -0,0 +1,34 @@
+/**
+ * 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.hadoop.hive.ql.security;
+
+import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
+import org.apache.hadoop.hive.ql.security.authorization.MetaStoreAuthzAPIAuthorizerEmbedOnly;
+
+/**
+ * Authorizer that prevents any authorization api call from being made. For use in testing.
+ */
+public class MetastoreAuthzAPIDisallowAuthorizer extends MetaStoreAuthzAPIAuthorizerEmbedOnly {
+  public static final String errMsg = "Metastore Authorization api invocation is disabled"
+      + " in this configuration.";
+
+  @Override
+  public void authorizeAuthorizationApiInvocation() throws AuthorizationException {
+      throw new AuthorizationException(errMsg);
+  }
+}

Modified: hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidatorForTest.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidatorForTest.java?rev=1626347&r1=1626346&r2=1626347&view=diff
==============================================================================
--- hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidatorForTest.java (original)
+++ hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidatorForTest.java Fri Sep 19 21:53:03 2014
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.securit
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactory;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
@@ -38,8 +39,9 @@ public class SQLStdHiveAuthorizationVali
 
   public SQLStdHiveAuthorizationValidatorForTest(HiveMetastoreClientFactory metastoreClientFactory,
       HiveConf conf, HiveAuthenticationProvider authenticator,
-      SQLStdHiveAccessControllerWrapper privController) {
-    super(metastoreClientFactory, conf, authenticator, privController);
+      SQLStdHiveAccessControllerWrapper privController, HiveAuthzSessionContext ctx)
+      throws HiveAuthzPluginException {
+    super(metastoreClientFactory, conf, authenticator, privController, ctx);
   }
 
   @Override

Modified: hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactoryForTest.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactoryForTest.java?rev=1626347&r1=1626346&r2=1626347&view=diff
==============================================================================
--- hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactoryForTest.java (original)
+++ hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactoryForTest.java Fri Sep 19 21:53:03 2014
@@ -37,7 +37,7 @@ public class SQLStdHiveAuthorizerFactory
     return new HiveAuthorizerImpl(
         privilegeManager,
         new SQLStdHiveAuthorizationValidatorForTest(metastoreClientFactory, conf, authenticator,
-            privilegeManager)
+            privilegeManager, ctx)
         );
   }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandUtil.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandUtil.java?rev=1626347&r1=1626346&r2=1626347&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandUtil.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandUtil.java Fri Sep 19 21:53:03 2014
@@ -21,6 +21,9 @@ package org.apache.hadoop.hive.ql.proces
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
@@ -31,6 +34,7 @@ import org.apache.hadoop.hive.ql.session
 import com.google.common.base.Joiner;
 
 class CommandUtil {
+  public static final Log LOG = LogFactory.getLog(CommandUtil.class);
 
   /**
    * Authorize command of given type and arguments
@@ -47,14 +51,19 @@ class CommandUtil {
       // ss can be null in unit tests
       return null;
     }
-    if (ss.isAuthorizationModeV2()) {
+
+    if (ss.isAuthorizationModeV2() &&
+        HiveConf.getBoolVar(ss.getConf(), HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)) {
+      String errMsg = "Error authorizing command " + command;
       try {
         authorizeCommandThrowEx(ss, type, command);
         // authorized to perform action
         return null;
       } catch (HiveAuthzPluginException e) {
+        LOG.error(errMsg, e);
         return CommandProcessorResponse.create(e);
       } catch (HiveAccessControlException e) {
+        LOG.error(errMsg, e);
         return CommandProcessorResponse.create(e);
       }
     }

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/DummyHiveAuthorizationValidator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/DummyHiveAuthorizationValidator.java?rev=1626347&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/DummyHiveAuthorizationValidator.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/DummyHiveAuthorizationValidator.java Fri Sep 19 21:53:03 2014
@@ -0,0 +1,45 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin.sqlstd;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizationValidator;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+
+/**
+ * A no-op HiveAuthorizationValidator for use from hive cli.
+ */
+public class DummyHiveAuthorizationValidator implements HiveAuthorizationValidator {
+
+  public static final Log LOG = LogFactory.getLog(DummyHiveAuthorizationValidator.class);
+
+  @Override
+  public void checkPrivileges(HiveOperationType hiveOpType, List<HivePrivilegeObject> inputHObjs,
+      List<HivePrivilegeObject> outputHObjs, HiveAuthzContext context)
+      throws HiveAuthzPluginException, HiveAccessControlException {
+    // no-op
+  }
+
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLAuthorizationUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLAuthorizationUtils.java?rev=1626347&r1=1626346&r2=1626347&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLAuthorizationUtils.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLAuthorizationUtils.java Fri Sep 19 21:53:03 2014
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.Database;
@@ -53,6 +54,8 @@ import org.apache.hadoop.hive.ql.metadat
 import org.apache.hadoop.hive.ql.security.authorization.AuthorizationUtils;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext.CLIENT_TYPE;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrincipal;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilege;
@@ -455,4 +458,23 @@ public class SQLAuthorizationUtils {
     return hivePrincipals;
   }
 
+  /**
+   * Change the session context based on configuration to aid in testing of sql
+   * std auth
+   *
+   * @param ctx
+   * @param conf
+   * @return
+   */
+  static HiveAuthzSessionContext applyTestSettings(HiveAuthzSessionContext ctx, HiveConf conf) {
+    if (conf.getBoolVar(ConfVars.HIVE_TEST_AUTHORIZATION_SQLSTD_HS2_MODE)
+        && ctx.getClientType() == CLIENT_TYPE.HIVECLI) {
+      // create new session ctx object with HS2 as client type
+      HiveAuthzSessionContext.Builder ctxBuilder = new HiveAuthzSessionContext.Builder(ctx);
+      ctxBuilder.setClientType(CLIENT_TYPE.HIVESERVER2);
+      return ctxBuilder.build();
+    }
+    return ctx;
+  }
+
 }

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdConfOnlyAuthorizerFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdConfOnlyAuthorizerFactory.java?rev=1626347&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdConfOnlyAuthorizerFactory.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdConfOnlyAuthorizerFactory.java Fri Sep 19 21:53:03 2014
@@ -0,0 +1,49 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin.sqlstd;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.Private;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
+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;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactory;
+
+/**
+ * Authorization class that can be used from hive cli, so that configuration
+ * in cli mode is set appropriately for SQL standards authorization.
+ * This ensures that new tables and views have proper privileges for the table/view owner.
+ *
+ * Uses DummyHiveAuthorizationValidator for no-op authorization checks. Authorization using
+ * sql standards based authorization mode can't be done securely with hive-cli, as hive-cli
+ * users have direct access to the file system.
+ */
+@Private
+public class SQLStdConfOnlyAuthorizerFactory implements HiveAuthorizerFactory {
+  @Override
+  public HiveAuthorizer createHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory,
+      HiveConf conf, HiveAuthenticationProvider authenticator, HiveAuthzSessionContext ctx) throws HiveAuthzPluginException {
+
+    SQLStdHiveAccessControllerWrapper privilegeManager =
+        new SQLStdHiveAccessControllerWrapper(metastoreClientFactory, conf, authenticator, ctx);
+    return new HiveAuthorizerImpl(privilegeManager, new DummyHiveAuthorizationValidator());
+  }
+}

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=1626347&r1=1626346&r2=1626347&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 Fri Sep 19 21:53:03 2014
@@ -90,42 +90,11 @@ public class SQLStdHiveAccessController 
       HiveAuthenticationProvider authenticator, HiveAuthzSessionContext ctx) throws HiveAuthzPluginException {
     this.metastoreClientFactory = metastoreClientFactory;
     this.authenticator = authenticator;
-    this.sessionCtx = applyTestSettings(ctx, conf);
-
-    assertHiveCliAuthDisabled(conf);
-    initUserRoles();
+    this.sessionCtx = SQLAuthorizationUtils.applyTestSettings(ctx, conf);
     LOG.info("Created SQLStdHiveAccessController for session context : " + sessionCtx);
   }
 
   /**
-   * Change the session context based on configuration to aid in testing of sql std auth
-   * @param ctx
-   * @param conf
-   * @return
-   */
-  private HiveAuthzSessionContext applyTestSettings(HiveAuthzSessionContext ctx, HiveConf conf) {
-    if(conf.getBoolVar(ConfVars.HIVE_TEST_AUTHORIZATION_SQLSTD_HS2_MODE) &&
-        ctx.getClientType() == CLIENT_TYPE.HIVECLI
-        ){
-      // create new session ctx object with HS2 as client type
-      HiveAuthzSessionContext.Builder ctxBuilder = new HiveAuthzSessionContext.Builder(ctx);
-      ctxBuilder.setClientType(CLIENT_TYPE.HIVESERVER2);
-      return ctxBuilder.build();
-    }
-    return ctx;
-  }
-
-  private void assertHiveCliAuthDisabled(HiveConf conf) throws HiveAuthzPluginException {
-    if (sessionCtx.getClientType() == CLIENT_TYPE.HIVECLI
-        && conf.getBoolVar(ConfVars.HIVE_AUTHORIZATION_ENABLED)) {
-      throw new HiveAuthzPluginException(
-          "SQL standards based authorization should not be enabled from hive cli"
-              + "Instead the use of storage based authorization in hive metastore is reccomended. Set "
-              + ConfVars.HIVE_AUTHORIZATION_ENABLED.varname + "=false to disable authz within cli");
-    }
-  }
-
-  /**
    * (Re-)initialize currentRoleNames if necessary.
    * @throws HiveAuthzPluginException
    */

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java?rev=1626347&r1=1626346&r2=1626347&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java Fri Sep 19 21:53:03 2014
@@ -25,12 +25,15 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.IMetaStoreClient;
 import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizationValidator;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext.CLIENT_TYPE;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactory;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrincipal;
@@ -44,16 +47,30 @@ public class SQLStdHiveAuthorizationVali
   private final HiveConf conf;
   private final HiveAuthenticationProvider authenticator;
   private final SQLStdHiveAccessControllerWrapper privController;
+  private final HiveAuthzSessionContext ctx;
   public static final Log LOG = LogFactory.getLog(SQLStdHiveAuthorizationValidator.class);
 
   public SQLStdHiveAuthorizationValidator(HiveMetastoreClientFactory metastoreClientFactory,
       HiveConf conf, HiveAuthenticationProvider authenticator,
-      SQLStdHiveAccessControllerWrapper privilegeManager) {
+      SQLStdHiveAccessControllerWrapper privilegeManager, HiveAuthzSessionContext ctx)
+      throws HiveAuthzPluginException {
 
     this.metastoreClientFactory = metastoreClientFactory;
     this.conf = conf;
     this.authenticator = authenticator;
     this.privController = privilegeManager;
+    this.ctx = SQLAuthorizationUtils.applyTestSettings(ctx, conf);
+    assertHiveCliAuthDisabled(conf);
+  }
+
+  private void assertHiveCliAuthDisabled(HiveConf conf) throws HiveAuthzPluginException {
+    if (ctx.getClientType() == CLIENT_TYPE.HIVECLI
+        && conf.getBoolVar(ConfVars.HIVE_AUTHORIZATION_ENABLED)) {
+      throw new HiveAuthzPluginException(
+          "SQL standards based authorization should not be enabled from hive cli"
+              + "Instead the use of storage based authorization in hive metastore is reccomended. Set "
+              + ConfVars.HIVE_AUTHORIZATION_ENABLED.varname + "=false to disable authz within cli");
+    }
   }
 
   @Override

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java?rev=1626347&r1=1626346&r2=1626347&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java Fri Sep 19 21:53:03 2014
@@ -37,7 +37,7 @@ public class SQLStdHiveAuthorizerFactory
     return new HiveAuthorizerImpl(
         privilegeManager,
         new SQLStdHiveAuthorizationValidator(metastoreClientFactory, conf, authenticator,
-            privilegeManager)
+            privilegeManager, ctx)
         );
   }
 }

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java?rev=1626347&r1=1626346&r2=1626347&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java Fri Sep 19 21:53:03 2014
@@ -25,6 +25,8 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.security.HadoopDefaultAuthenticator;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.DisallowTransformHook;
+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.HiveAuthzPluginException;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext.Builder;
@@ -77,8 +79,9 @@ public class TestSQLStdHiveAccessControl
     HiveConf processedConf = new HiveConf();
     processedConf.setBoolVar(ConfVars.HIVE_AUTHORIZATION_ENABLED, true);
     try {
-      SQLStdHiveAccessController accessController = new SQLStdHiveAccessController(null,
-          processedConf, new HadoopDefaultAuthenticator(), getCLISessionCtx());
+      HiveAuthorizerFactory authorizerFactory = new SQLStdHiveAuthorizerFactory();
+      HiveAuthorizer authorizer = authorizerFactory.createHiveAuthorizer(null, processedConf,
+          new HadoopDefaultAuthenticator(), getCLISessionCtx());
       fail("Exception expected");
     } catch (HiveAuthzPluginException e) {
       assertTrue(e.getMessage().contains(

Modified: hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_createtab.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_createtab.q?rev=1626347&r1=1626346&r2=1626347&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_createtab.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_createtab.q Fri Sep 19 21:53:03 2014
@@ -1,6 +1,5 @@
-set hive.test.authz.sstd.hs2.mode=true;
 set hive.users.in.admin.role=hive_admin_user;
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set user.name=hive_test_user;
 

Added: hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_createtab_noauthzapi.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_createtab_noauthzapi.q?rev=1626347&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_createtab_noauthzapi.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_createtab_noauthzapi.q Fri Sep 19 21:53:03 2014
@@ -0,0 +1,12 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.metastore.pre.event.listeners=org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventListener;
+set hive.security.metastore.authorization.manager=org.apache.hadoop.hive.ql.security.MetastoreAuthzAPIDisallowAuthorizer;
+set user.name=hive_test_user;
+
+-- verify that sql std auth can be set as the authorizer with hive cli, while metastore authorization api calls are disabled (for cli)
+
+create table t_cli(i int);
+
+create view v_cli (i) as select i from t_cli;

Added: hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_nonsql.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_nonsql.q?rev=1626347&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_nonsql.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_nonsql.q Fri Sep 19 21:53:03 2014
@@ -0,0 +1,29 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.enabled=false;
+
+-- Verify that dfs,compile,add,delete commands can be run from hive cli, and no authorization checks happen when auth is diabled
+
+use default;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_admin_almighty1;
+dfs -ls ${system:test.tmp.dir}/a_admin_almighty1;
+
+create table a_table1(a int, b int);
+add jar ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar;
+alter table a_table1 set serde 'org.apache.hadoop.hive.serde2.TestSerDe' with serdeproperties('s1'='9');
+drop table a_table;
+
+delete jar ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar;
+
+compile `import org.apache.hadoop.hive.ql.exec.UDF \;
+public class Pyth extends UDF {
+  public double evaluate(double a, double b){
+    return Math.sqrt((a*a) + (b*b)) \;
+  }
+} `AS GROOVY NAMED Pyth.groovy;
+CREATE TEMPORARY FUNCTION Pyth as 'Pyth';
+
+SELECT Pyth(3,4) FROM src tablesample (1 rows);
+
+DROP TEMPORARY FUNCTION Pyth;
+

Added: hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_stdconfigauth.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_stdconfigauth.q?rev=1626347&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_stdconfigauth.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_cli_stdconfigauth.q Fri Sep 19 21:53:03 2014
@@ -0,0 +1,10 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdConfOnlyAuthorizerFactory;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+-- verify that SQLStdConfOnlyAuthorizerFactory as the authorizer factory with hive cli, with hive.security.authorization.enabled=true
+-- authorization verification would be just no-op
+
+create table t_cli(i int);
+describe t_cli;

Added: hive/trunk/ql/src/test/results/clientpositive/authorization_cli_createtab_noauthzapi.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/authorization_cli_createtab_noauthzapi.q.out?rev=1626347&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/authorization_cli_createtab_noauthzapi.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/authorization_cli_createtab_noauthzapi.q.out Fri Sep 19 21:53:03 2014
@@ -0,0 +1,22 @@
+PREHOOK: query: -- verify that sql std auth can be set as the authorizer with hive cli, while metastore authorization api calls are disabled (for cli)
+
+create table t_cli(i int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t_cli
+POSTHOOK: query: -- verify that sql std auth can be set as the authorizer with hive cli, while metastore authorization api calls are disabled (for cli)
+
+create table t_cli(i int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t_cli
+PREHOOK: query: create view v_cli (i) as select i from t_cli
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: default@t_cli
+PREHOOK: Output: database:default
+PREHOOK: Output: default@v_cli
+POSTHOOK: query: create view v_cli (i) as select i from t_cli
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: default@t_cli
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@v_cli

Added: hive/trunk/ql/src/test/results/clientpositive/authorization_cli_nonsql.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/authorization_cli_nonsql.q.out?rev=1626347&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/authorization_cli_nonsql.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/authorization_cli_nonsql.q.out Fri Sep 19 21:53:03 2014
@@ -0,0 +1,51 @@
+PREHOOK: query: -- Verify that dfs,compile,add,delete commands can be run from hive cli, and no authorization checks happen when auth is diabled
+
+use default
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:default
+POSTHOOK: query: -- Verify that dfs,compile,add,delete commands can be run from hive cli, and no authorization checks happen when auth is diabled
+
+use default
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:default
+PREHOOK: query: create table a_table1(a int, b int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@a_table1
+POSTHOOK: query: create table a_table1(a int, b int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@a_table1
+PREHOOK: query: alter table a_table1 set serde 'org.apache.hadoop.hive.serde2.TestSerDe' with serdeproperties('s1'='9')
+PREHOOK: type: ALTERTABLE_SERIALIZER
+PREHOOK: Input: default@a_table1
+PREHOOK: Output: default@a_table1
+POSTHOOK: query: alter table a_table1 set serde 'org.apache.hadoop.hive.serde2.TestSerDe' with serdeproperties('s1'='9')
+POSTHOOK: type: ALTERTABLE_SERIALIZER
+POSTHOOK: Input: default@a_table1
+POSTHOOK: Output: default@a_table1
+PREHOOK: query: drop table a_table
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table a_table
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TEMPORARY FUNCTION Pyth as 'Pyth'
+PREHOOK: type: CREATEFUNCTION
+PREHOOK: Output: pyth
+POSTHOOK: query: CREATE TEMPORARY FUNCTION Pyth as 'Pyth'
+POSTHOOK: type: CREATEFUNCTION
+POSTHOOK: Output: pyth
+PREHOOK: query: SELECT Pyth(3,4) FROM src tablesample (1 rows)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT Pyth(3,4) FROM src tablesample (1 rows)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+5.0
+PREHOOK: query: DROP TEMPORARY FUNCTION Pyth
+PREHOOK: type: DROPFUNCTION
+PREHOOK: Output: Pyth
+POSTHOOK: query: DROP TEMPORARY FUNCTION Pyth
+POSTHOOK: type: DROPFUNCTION
+POSTHOOK: Output: Pyth

Added: hive/trunk/ql/src/test/results/clientpositive/authorization_cli_stdconfigauth.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/authorization_cli_stdconfigauth.q.out?rev=1626347&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/authorization_cli_stdconfigauth.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/authorization_cli_stdconfigauth.q.out Fri Sep 19 21:53:03 2014
@@ -0,0 +1,21 @@
+PREHOOK: query: -- verify that SQLStdConfOnlyAuthorizerFactory as the authorizer factory with hive cli, with hive.security.authorization.enabled=true
+-- authorization verification would be just no-op
+
+create table t_cli(i int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t_cli
+POSTHOOK: query: -- verify that SQLStdConfOnlyAuthorizerFactory as the authorizer factory with hive cli, with hive.security.authorization.enabled=true
+-- authorization verification would be just no-op
+
+create table t_cli(i int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t_cli
+PREHOOK: query: describe t_cli
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@t_cli
+POSTHOOK: query: describe t_cli
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@t_cli
+i                   	int