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

svn commit: r1585791 [1/2] - in /hive/branches/branch-0.13: common/src/java/org/apache/hadoop/hive/conf/ itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/ itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/ ites...

Author: hashutosh
Date: Tue Apr  8 17:38:20 2014
New Revision: 1585791

URL: http://svn.apache.org/r1585791
Log:
HIVE-6846 : allow safe set commands with sql standard authorization (Thejas Nair via Ashutosh Chauhan)

Added:
    hive/branches/branch-0.13/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/
    hive/branches/branch-0.13/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/
    hive/branches/branch-0.13/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/
    hive/branches/branch-0.13/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessControllerForTest.java
    hive/branches/branch-0.13/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactoryForTest.java
    hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/
    hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessController.java
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_dfs.q
    hive/branches/branch-0.13/ql/src/test/results/clientnegative/authorization_dfs.q.out
Modified:
    hive/branches/branch-0.13/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
    hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestSessionUserName.java
    hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_addjar.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_addpartition.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_func1.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_func2.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_macro1.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_createview.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_ctas.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_disallow_transform.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_droppartition.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_role_cycles1.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_role_cycles2.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_role_grant.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_select.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_select_view.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_truncate.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_uri_createdb.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_uri_index.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_uri_insert.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_uri_load_data.q
    hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorize_create_tbl.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/authorization_1_sql_std.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/authorization_admin_almighty1.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/authorization_create_func1.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/authorization_create_macro1.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/authorization_owner_actions.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/authorization_owner_actions_db.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/authorization_role_grant1.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/authorization_role_grant2.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/authorization_view_sqlstd.q
    hive/branches/branch-0.13/ql/src/test/results/clientnegative/authorization_addjar.q.out
    hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
    hive/branches/branch-0.13/service/src/java/org/apache/hive/service/cli/session/SessionManager.java

Modified: hive/branches/branch-0.13/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/branches/branch-0.13/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Tue Apr  8 17:38:20 2014
@@ -18,24 +18,37 @@
 
 package org.apache.hadoop.hive.conf;
 
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import javax.security.auth.login.LoginException;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Shell;
 import org.apache.hive.common.HiveCompat;
 
-import javax.security.auth.login.LoginException;
-import java.io.*;
-import java.net.URL;
-import java.util.*;
-import java.util.Map.Entry;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
 /**
  * Hive Configuration.
  */
@@ -52,6 +65,9 @@ public class HiveConf extends Configurat
   private static final Map<String, ConfVars> vars = new HashMap<String, ConfVars>();
   private final List<String> restrictList = new ArrayList<String>();
 
+  private boolean isWhiteListRestrictionEnabled = false;
+  private final List<String> modWhiteList = new ArrayList<String>();
+
   static {
     ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
     if (classLoader == null) {
@@ -129,6 +145,7 @@ public class HiveConf extends Configurat
       HiveConf.ConfVars.HIVE_TXN_MAX_OPEN_BATCH,
       };
 
+
   /**
    * dbVars are the parameters can be set per database. If these
    * parameters are set as a database property, when switching to that
@@ -801,6 +818,11 @@ public class HiveConf extends Configurat
     HIVE_AUTHORIZATION_TABLE_OWNER_GRANTS("hive.security.authorization.createtable.owner.grants",
         ""),
 
+    // if this is not set default value is added by sql standard authorizer.
+    // Default value can't be set in this constructor as it would refer names in other ConfVars
+    // whose constructor would not have been called
+    HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST("hive.security.authorization.sqlstd.confwhitelist", ""),
+
     // Print column names in output
     HIVE_CLI_PRINT_HEADER("hive.cli.print.header", false),
 
@@ -995,6 +1017,7 @@ public class HiveConf extends Configurat
     // Check if a plan contains a Cross Product.
     // If there is one, output a warning to the Session's console.
     HIVE_CHECK_CROSS_PRODUCT("hive.exec.check.crossproducts", true),
+
     ;
 
     public final String varname;
@@ -1169,8 +1192,15 @@ public class HiveConf extends Configurat
   }
 
   public void verifyAndSet(String name, String value) throws IllegalArgumentException {
+    if (isWhiteListRestrictionEnabled) {
+      if (!modWhiteList.contains(name)) {
+        throw new IllegalArgumentException("Cannot modify " + name + " at runtime. "
+            + "It is not in list of params that are allowed to be modified at runtime");
+      }
+    }
     if (restrictList.contains(name)) {
-      throw new IllegalArgumentException("Cannot modify " + name + " at runtime");
+      throw new IllegalArgumentException("Cannot modify " + name + " at runtime. It is in the list"
+          + "of parameters that can't be modified at runtime");
     }
     set(name, value);
   }
@@ -1598,6 +1628,29 @@ public class HiveConf extends Configurat
   }
 
   /**
+   * Set if whitelist check is enabled for parameter modification
+   *
+   * @param isEnabled
+   */
+  @LimitedPrivate(value = { "Currently only for use by HiveAuthorizer" })
+  public void setIsModWhiteListEnabled(boolean isEnabled) {
+    this.isWhiteListRestrictionEnabled = isEnabled;
+  }
+
+  /**
+   * Add config parameter name to whitelist of parameters that can be modified
+   *
+   * @param paramname
+   */
+  @LimitedPrivate(value = { "Currently only for use by HiveAuthorizer" })
+  public void addToModifiableWhiteList(String paramname) {
+    if (paramname == null) {
+      return;
+    }
+    modWhiteList.add(paramname);
+  }
+
+  /**
    * Add the HIVE_CONF_RESTRICTED_LIST values to restrictList,
    * including HIVE_CONF_RESTRICTED_LIST itself
    */

Modified: hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java (original)
+++ hive/branches/branch-0.13/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java Tue Apr  8 17:38:20 2014
@@ -112,4 +112,30 @@ public class TestJdbcWithSQLAuthorizatio
     return DriverManager.getConnection(miniHS2.getJdbcURL(), userName, "bar");
   }
 
+  @Test
+  public void testAllowedCommands() throws Exception {
+
+    // 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");
+      boolean caughtException = false;
+      Statement stmt = hs2Conn.createStatement();
+      // create tables
+      try {
+        stmt.execute("dfs -ls /tmp/");
+      } catch (SQLException e){
+        caughtException = true;
+        assertTrue("Checking error message content",
+            e.getMessage().contains("Insufficient privileges to execute"));
+      }
+      finally {
+        stmt.close();
+        hs2Conn.close();
+      }
+      assertTrue("Exception expected ", caughtException);
+    }
+  }
+
 }

Added: hive/branches/branch-0.13/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessControllerForTest.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessControllerForTest.java?rev=1585791&view=auto
==============================================================================
--- hive/branches/branch-0.13/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessControllerForTest.java (added)
+++ hive/branches/branch-0.13/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessControllerForTest.java Tue Apr  8 17:38:20 2014
@@ -0,0 +1,54 @@
+/**
+ * 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.classification.InterfaceAudience.Private;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactory;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAccessController;
+
+/**
+ * Extends SQLStdHiveAccessController to relax the restriction of not being able to run dfs
+ * and set commands, so that it is easy to test using .q file tests.
+ * To be used for testing purposes only!
+ */
+@Private
+public class SQLStdHiveAccessControllerForTest extends SQLStdHiveAccessController {
+
+  SQLStdHiveAccessControllerForTest(HiveMetastoreClientFactory metastoreClientFactory, HiveConf conf,
+      HiveAuthenticationProvider authenticator) throws HiveAuthzPluginException {
+    super(metastoreClientFactory, conf, authenticator);
+  }
+
+
+  @Override
+  public void applyAuthorizationConfigPolicy(HiveConf hiveConf) {
+    super.applyAuthorizationConfigPolicy(hiveConf);
+
+    // allow set and dfs commands
+    hiveConf.setVar(ConfVars.HIVE_SECURITY_COMMAND_WHITELIST, "set,dfs");
+
+    // remove restrictions on the variables that can be set using set command
+    hiveConf.setIsModWhiteListEnabled(false);
+
+  }
+
+}

Added: hive/branches/branch-0.13/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactoryForTest.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactoryForTest.java?rev=1585791&view=auto
==============================================================================
--- hive/branches/branch-0.13/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactoryForTest.java (added)
+++ hive/branches/branch-0.13/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactoryForTest.java Tue Apr  8 17:38:20 2014
@@ -0,0 +1,42 @@
+/**
+ * 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.HiveMetastoreClientFactory;
+
+@Private
+public class SQLStdHiveAuthorizerFactoryForTest implements HiveAuthorizerFactory{
+  @Override
+  public HiveAuthorizer createHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory,
+      HiveConf conf, HiveAuthenticationProvider authenticator) throws HiveAuthzPluginException {
+    SQLStdHiveAccessController privilegeManager =
+        new SQLStdHiveAccessControllerForTest(metastoreClientFactory, conf, authenticator);
+    return new HiveAuthorizerImpl(
+        privilegeManager,
+        new SQLStdHiveAuthorizationValidator(metastoreClientFactory, conf, authenticator,
+            privilegeManager)
+        );
+  }
+}

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java Tue Apr  8 17:38:20 2014
@@ -28,10 +28,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.Driver;
-import org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider;
-import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizerFactory;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
 /**
@@ -61,18 +58,8 @@ public final class CommandProcessorFacto
       conf = new HiveConf();
     }
     Set<String> availableCommands = new HashSet<String>();
-    if (!HiveAuthorizerFactory.class.isAssignableFrom
-      (conf.getClass(ConfVars.HIVE_AUTHORIZATION_MANAGER.varname,DefaultHiveAuthorizationProvider.class))) {
-      // we are not on authV2, add processors.
-      for (String availableCommand : conf.getVar(HiveConf.ConfVars.HIVE_SECURITY_COMMAND_WHITELIST).split(",")) {
-        availableCommands.add(availableCommand.toLowerCase().trim());
-      }
-    }
-
-    if (conf.getBoolVar(ConfVars.HIVE_IN_TEST)) {
-      // because test case uses these.
-      availableCommands.add("set");
-      availableCommands.add("dfs");
+    for (String availableCommand : conf.getVar(HiveConf.ConfVars.HIVE_SECURITY_COMMAND_WHITELIST).split(",")) {
+      availableCommands.add(availableCommand.toLowerCase().trim());
     }
     if (!availableCommands.contains(cmd[0].trim().toLowerCase())) {
       throw new SQLException("Insufficient privileges to execute " + cmd[0], "42000");

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java Tue Apr  8 17:38:20 2014
@@ -21,6 +21,7 @@ import java.util.List;
 
 import org.apache.hadoop.hive.common.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.hive.common.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.hive.conf.HiveConf;
 
 /**
  * Interface that is invoked by access control commands, including grant/revoke role/privileges,
@@ -68,4 +69,6 @@ public interface HiveAccessController {
 
   List<HiveRoleGrant> getRoleGrantInfoForPrincipal(HivePrincipal principal) throws HiveAuthzPluginException,
       HiveAccessControlException;
+
+  void applyAuthorizationConfigPolicy(HiveConf hiveConf);
 }

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java Tue Apr  8 17:38:20 2014
@@ -21,6 +21,7 @@ import java.util.List;
 
 import org.apache.hadoop.hive.common.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.hive.common.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
 
 /**
@@ -171,9 +172,26 @@ public interface HiveAuthorizer {
   List<HivePrivilegeInfo> showPrivileges(HivePrincipal principal, HivePrivilegeObject privObj)
       throws HiveAuthzPluginException, HiveAccessControlException;
 
+  /**
+   * Set the current role to roleName argument
+   * @param roleName
+   * @throws HiveAccessControlException
+   * @throws HiveAuthzPluginException
+   */
   void setCurrentRole(String roleName) throws HiveAccessControlException, HiveAuthzPluginException;
 
+  /**
+   * @return List having names of current roles
+   * @throws HiveAuthzPluginException
+   */
   List<String> getCurrentRoleNames() throws HiveAuthzPluginException;
 
+  /**
+   * Modify the given HiveConf object to configure authorization related parameters
+   * or other parameters related to hive security
+   * @param hiveConf
+   */
+  public void applyAuthorizationConfigPolicy(HiveConf hiveConf);
+
 }
 

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java Tue Apr  8 17:38:20 2014
@@ -21,6 +21,7 @@ import java.util.List;
 
 import org.apache.hadoop.hive.common.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.hive.common.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.hive.conf.HiveConf;
 
 /**
  * Convenience implementation of HiveAuthorizer.
@@ -120,4 +121,9 @@ public class HiveAuthorizerImpl implemen
       throws HiveAuthzPluginException, HiveAccessControlException {
     return accessController.getRoleGrantInfoForPrincipal(principal);
   }
+
+  @Override
+  public void applyAuthorizationConfigPolicy(HiveConf hiveConf) {
+    accessController.applyAuthorizationConfigPolicy(hiveConf);
+  }
 }

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java Tue Apr  8 17:38:20 2014
@@ -24,8 +24,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.HiveMetaStore;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest;
@@ -43,6 +46,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.AuthorizationUtils;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.DisallowTransformHook;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessController;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
@@ -55,6 +59,7 @@ import org.apache.hadoop.hive.ql.securit
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveRoleGrant;
 import org.apache.thrift.TException;
 
+import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableSet;
 
 /**
@@ -76,8 +81,9 @@ public class SQLStdHiveAccessController 
       + "have it as current role, for this action.";
   private final String HAS_ADMIN_PRIV_MSG = "grantor need to have ADMIN privileges on role being"
       + " granted and have it as a current role for this action.";
+  public static final Log LOG = LogFactory.getLog(SQLStdHiveAccessController.class);
 
-  SQLStdHiveAccessController(HiveMetastoreClientFactory metastoreClientFactory, HiveConf conf,
+  public SQLStdHiveAccessController(HiveMetastoreClientFactory metastoreClientFactory, HiveConf conf,
       HiveAuthenticationProvider authenticator) throws HiveAuthzPluginException {
     this.metastoreClientFactory = metastoreClientFactory;
     this.authenticator = authenticator;
@@ -523,4 +529,100 @@ public class SQLStdHiveAccessController 
     }
   }
 
+
+  /**
+   * Default list of modifiable config parameters for sql standard authorization
+   */
+  static final String [] defaultModWhiteListSqlStdAuth = new String [] {
+      ConfVars.BYTESPERREDUCER.varname,
+      ConfVars.MAXREDUCERS.varname,
+      ConfVars.HIVEMAPSIDEAGGREGATE.varname,
+      ConfVars.HIVEMAPAGGRHASHMEMORY.varname,
+      ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD.varname,
+      ConfVars.HIVEMAPAGGRHASHMINREDUCTION.varname,
+      ConfVars.HIVEGROUPBYSKEW.varname,
+      ConfVars.HIVE_OPTIMIZE_MULTI_GROUPBY_COMMON_DISTINCTS.varname,
+      ConfVars.HIVEOPTGBYUSINGINDEX.varname,
+      ConfVars.HIVEOPTPPD.varname,
+      ConfVars.HIVEOPTPPD_STORAGE.varname,
+      ConfVars.HIVEOPTPPD_STORAGE.varname,
+      ConfVars.HIVEPPDRECOGNIZETRANSITIVITY.varname,
+      ConfVars.HIVEOPTGROUPBY.varname,
+      ConfVars.HIVEOPTSORTDYNAMICPARTITION.varname,
+      ConfVars.HIVE_OPTIMIZE_SKEWJOIN_COMPILETIME.varname,
+      ConfVars.HIVE_OPTIMIZE_UNION_REMOVE.varname,
+      ConfVars.HIVEMULTIGROUPBYSINGLEREDUCER.varname,
+      ConfVars.HIVE_MAP_GROUPBY_SORT.varname,
+      ConfVars.HIVE_MAP_GROUPBY_SORT_TESTMODE.varname,
+      ConfVars.HIVESKEWJOIN.varname,
+      ConfVars.HIVE_OPTIMIZE_SKEWJOIN_COMPILETIME.varname,
+      ConfVars.HIVEMAPREDMODE.varname,
+      ConfVars.HIVEENFORCEBUCKETMAPJOIN.varname,
+      ConfVars.COMPRESSRESULT.varname,
+      ConfVars.COMPRESSINTERMEDIATE.varname,
+      ConfVars.EXECPARALLEL.varname,
+      ConfVars.EXECPARALLETHREADNUMBER.varname,
+      ConfVars.EXECPARALLETHREADNUMBER.varname,
+      ConfVars.HIVEROWOFFSET.varname,
+      ConfVars.HIVEMERGEMAPFILES.varname,
+      ConfVars.HIVEMERGEMAPREDFILES.varname,
+      ConfVars.HIVEMERGETEZFILES.varname,
+      ConfVars.HIVEIGNOREMAPJOINHINT.varname,
+      ConfVars.HIVECONVERTJOIN.varname,
+      ConfVars.HIVECONVERTJOINNOCONDITIONALTASK.varname,
+      ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD.varname,
+      ConfVars.HIVECONVERTJOINUSENONSTAGED.varname,
+      ConfVars.HIVECONVERTJOINNOCONDITIONALTASK.varname,
+      ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD.varname,
+      ConfVars.HIVECONVERTJOINUSENONSTAGED.varname,
+      ConfVars.HIVEENFORCEBUCKETING.varname,
+      ConfVars.HIVEENFORCESORTING.varname,
+      ConfVars.HIVEENFORCESORTMERGEBUCKETMAPJOIN.varname,
+      ConfVars.HIVE_AUTO_SORTMERGE_JOIN.varname,
+      ConfVars.HIVE_EXECUTION_ENGINE.varname,
+      ConfVars.HIVE_VECTORIZATION_ENABLED.varname,
+      ConfVars.HIVEMAPJOINUSEOPTIMIZEDKEYS.varname,
+      ConfVars.HIVEMAPJOINLAZYHASHTABLE.varname,
+      ConfVars.HIVE_CHECK_CROSS_PRODUCT.varname,
+      ConfVars.HIVE_COMPAT.varname,
+      ConfVars.DYNAMICPARTITIONINGMODE.varname,
+      "mapred.reduce.tasks",
+      "mapred.output.compression.codec",
+      "mapred.map.output.compression.codec",
+      "mapreduce.job.reduce.slowstart.completedmaps",
+      "mapreduce.job.queuename",
+  };
+
+  @Override
+  public void applyAuthorizationConfigPolicy(HiveConf hiveConf) {
+    // grant all privileges for table to its owner
+    hiveConf.setVar(ConfVars.HIVE_AUTHORIZATION_TABLE_OWNER_GRANTS, "INSERT,SELECT,UPDATE,DELETE");
+
+    // Configure PREEXECHOOKS with DisallowTransformHook to disallow transform queries
+    String hooks = hiveConf.getVar(ConfVars.PREEXECHOOKS).trim();
+    if (hooks.isEmpty()) {
+      hooks = DisallowTransformHook.class.getName();
+    } else {
+      hooks = hooks + "," +DisallowTransformHook.class.getName();
+    }
+    LOG.debug("Configuring hooks : " + hooks);
+    hiveConf.setVar(ConfVars.PREEXECHOOKS, hooks);
+
+    // set security command list to only allow set command
+    hiveConf.setVar(ConfVars.HIVE_SECURITY_COMMAND_WHITELIST, "set");
+
+    // restrict the variables that can be set using set command to a list in whitelist
+    hiveConf.setIsModWhiteListEnabled(true);
+    String whiteListParamsStr = hiveConf.getVar(ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST);
+    if (whiteListParamsStr == null || whiteListParamsStr.trim().equals("")){
+      // set the default configs in whitelist
+      whiteListParamsStr = Joiner.on(",").join(defaultModWhiteListSqlStdAuth);
+      hiveConf.setVar(ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST, whiteListParamsStr);
+    }
+    for(String whiteListParam : whiteListParamsStr.split(",")){
+      hiveConf.addToModifiableWhiteList(whiteListParam);
+    }
+
+  }
+
 }

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java Tue Apr  8 17:38:20 2014
@@ -24,7 +24,6 @@ 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.metastore.HiveMetaStore;
 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;
@@ -43,7 +42,7 @@ public class SQLStdHiveAuthorizationVali
   private final HiveConf conf;
   private final HiveAuthenticationProvider authenticator;
   private final SQLStdHiveAccessController privController;
-  public static final Log LOG = LogFactory.getLog(HiveMetaStore.class);
+  public static final Log LOG = LogFactory.getLog(SQLStdHiveAuthorizationValidator.class);
 
   public SQLStdHiveAuthorizationValidator(HiveMetastoreClientFactory metastoreClientFactory,
       HiveConf conf, HiveAuthenticationProvider authenticator,

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java Tue Apr  8 17:38:20 2014
@@ -57,7 +57,6 @@ import org.apache.hadoop.hive.ql.metadat
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
 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;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizerFactory;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactoryImpl;
@@ -160,6 +159,9 @@ public class SessionState {
 
   private String currentDatabase;
 
+  private final String CONFIG_AUTHZ_SETTINGS_APPLIED_MARKER =
+      "hive.internal.ss.authz.settings.applied.marker";
+
   /**
    * Lineage state.
    */
@@ -370,34 +372,26 @@ public class SessionState {
     }
 
     try {
-      authenticator = HiveUtils.getAuthenticator(getConf(),
+      authenticator = HiveUtils.getAuthenticator(conf,
           HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER);
       authenticator.setSessionState(this);
 
-      authorizer = HiveUtils.getAuthorizeProviderManager(getConf(),
+      authorizer = HiveUtils.getAuthorizeProviderManager(conf,
           HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, authenticator, true);
 
       if (authorizer == null) {
         // if it was null, the new authorization plugin must be specified in
         // config
-        HiveAuthorizerFactory authorizerFactory = HiveUtils.getAuthorizerFactory(getConf(),
+        HiveAuthorizerFactory authorizerFactory = HiveUtils.getAuthorizerFactory(conf,
             HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER);
 
         authorizerV2 = authorizerFactory.createHiveAuthorizer(new HiveMetastoreClientFactoryImpl(),
-            getConf(), authenticator);
-        // grant all privileges for table to its owner
-        getConf().setVar(ConfVars.HIVE_AUTHORIZATION_TABLE_OWNER_GRANTS, "INSERT,SELECT,UPDATE,DELETE");
-        String hooks = getConf().getVar(ConfVars.PREEXECHOOKS).trim();
-        if (hooks.isEmpty()) {
-          hooks = DisallowTransformHook.class.getName();
-        } else {
-          hooks = hooks + "," +DisallowTransformHook.class.getName();
-        }
-        LOG.debug("Configuring hooks : " + hooks);
-        getConf().setVar(ConfVars.PREEXECHOOKS, hooks);
-      }
+            conf, authenticator);
 
-      createTableGrants = CreateTableAutomaticGrant.create(getConf());
+        authorizerV2.applyAuthorizationConfigPolicy(conf);
+        // create the create table grants with new config
+        createTableGrants = CreateTableAutomaticGrant.create(conf);
+      }
 
     } catch (HiveException e) {
       throw new RuntimeException(e);
@@ -1011,4 +1005,28 @@ public class SessionState {
     return userName;
   }
 
+  /**
+   * If authorization mode is v2, then pass it through authorizer so that it can apply
+   * any security configuration changes.
+   * @param hiveConf
+   * @return
+   * @throws HiveException
+   */
+  public void applyAuthorizationPolicy() throws HiveException {
+    if(!isAuthorizationModeV2()){
+      // auth v1 interface does not have this functionality
+      return;
+    }
+
+    // avoid processing the same config multiple times, check marker
+    if (conf.get(CONFIG_AUTHZ_SETTINGS_APPLIED_MARKER, "").equals(Boolean.TRUE.toString())) {
+      return;
+    }
+
+    authorizerV2.applyAuthorizationConfigPolicy(conf);
+    // set a marker that this conf has been processed.
+    conf.set(CONFIG_AUTHZ_SETTINGS_APPLIED_MARKER, Boolean.TRUE.toString());
+
+  }
+
 }

Modified: hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestSessionUserName.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestSessionUserName.java?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestSessionUserName.java (original)
+++ hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestSessionUserName.java Tue Apr  8 17:38:20 2014
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.metastore.
 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.HiveAccessController;
 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;
@@ -112,8 +113,9 @@ public class TestSessionUserName {
     public HiveAuthorizer createHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory,
         HiveConf conf, HiveAuthenticationProvider authenticator) {
       username = authenticator.getUserName();
-      return new HiveAuthorizerImpl(null, null);
+      HiveAccessController acontroller = Mockito.mock(HiveAccessController.class);
+      return new HiveAuthorizerImpl(acontroller, null);
     }
-  }
 
+  }
 }

Modified: hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java (original)
+++ hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java Tue Apr  8 17:38:20 2014
@@ -23,7 +23,6 @@ import java.sql.SQLException;
 import junit.framework.Assert;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.junit.Before;
 import org.junit.Test;
@@ -55,7 +54,6 @@ public class TestCommandProcessorFactory
       String cmd = command.name().toLowerCase();
       Assert.assertNotNull("Cmd " + cmd + " not return null", CommandProcessorFactory.getForHiveCommand(new String[]{cmd}, conf));
     }
-    conf.setBoolVar(ConfVars.HIVE_IN_TEST, false);
     conf.set(HiveConf.ConfVars.HIVE_SECURITY_COMMAND_WHITELIST.toString(), "");
     for (HiveCommand command : HiveCommand.values()) {
       String cmd = command.name();
@@ -67,6 +65,5 @@ public class TestCommandProcessorFactory
         Assert.assertEquals("42000", e.getSQLState());
       }
     }
-    conf.setBoolVar(ConfVars.HIVE_IN_TEST, true);
   }
 }

Added: hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessController.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessController.java?rev=1585791&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessController.java (added)
+++ hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessController.java Tue Apr  8 17:38:20 2014
@@ -0,0 +1,121 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+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.HiveAuthzPluginException;
+import org.junit.Test;
+
+import com.google.common.base.Joiner;
+
+/**
+ * Test SQLStdHiveAccessController
+ */
+public class TestSQLStdHiveAccessController {
+
+  /**
+   * Test if SQLStdHiveAccessController is applying configuration security
+   * policy on hiveconf correctly
+   *
+   * @throws HiveAuthzPluginException
+   */
+  @Test
+  public void checkConfigProcessing() throws HiveAuthzPluginException {
+    HiveConf processedConf = new HiveConf();
+
+    SQLStdHiveAccessController accessController = new SQLStdHiveAccessController(null,
+        processedConf, new HadoopDefaultAuthenticator());
+    accessController.applyAuthorizationConfigPolicy(processedConf);
+
+    // check that unsafe commands have been disabled
+    assertEquals("only set command should be allowed",
+        processedConf.getVar(ConfVars.HIVE_SECURITY_COMMAND_WHITELIST), "set");
+
+    // check that hook to disable transforms has been added
+    assertTrue("Check for transform query disabling hook",
+        processedConf.getVar(ConfVars.PREEXECHOOKS).contains(DisallowTransformHook.class.getName()));
+
+    verifyParamSettability(SQLStdHiveAccessController.defaultModWhiteListSqlStdAuth, processedConf);
+
+  }
+
+  /**
+   * Verify that params in settableParams can be modified, and other random ones can't be modified
+   * @param settableParams
+   * @param processedConf
+   */
+  private void verifyParamSettability(String [] settableParams, HiveConf processedConf) {
+    // verify that the whitlelist params can be set
+    for (String param : settableParams) {
+      try {
+        processedConf.verifyAndSet(param, "dummy");
+      } catch (IllegalArgumentException e) {
+        fail("Unable to set value for parameter in whitelist " + param + " " + e);
+      }
+    }
+
+    // verify that non whitelist params can't be set
+    assertConfModificationException(processedConf, "dummy.param");
+    // does not make sense to have any of the metastore config variables to be
+    // modifiable
+    for (ConfVars metaVar : HiveConf.metaVars) {
+      assertConfModificationException(processedConf, metaVar.varname);
+    }
+  }
+
+  /**
+   * Test that modifying HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST config works
+   * @throws HiveAuthzPluginException
+   */
+  @Test
+  public void checkConfigProcessingCustomSetWhitelist() throws HiveAuthzPluginException {
+
+    HiveConf processedConf = new HiveConf();
+    // add custom value, including one from the default, one new one
+    String [] settableParams = {SQLStdHiveAccessController.defaultModWhiteListSqlStdAuth[0], "abcs.dummy.test.param"};
+    processedConf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST,
+        Joiner.on(",").join(settableParams));
+
+
+    SQLStdHiveAccessController accessController = new SQLStdHiveAccessController(null,
+        processedConf, new HadoopDefaultAuthenticator());
+    accessController.applyAuthorizationConfigPolicy(processedConf);
+    verifyParamSettability(settableParams, processedConf);
+
+
+  }
+
+
+  private void assertConfModificationException(HiveConf processedConf, String param) {
+    boolean caughtEx = false;
+    try {
+      processedConf.verifyAndSet(param, "dummy");
+    } catch (IllegalArgumentException e) {
+      caughtEx = true;
+    }
+    assertTrue("Exception should be thrown while modifying the param " + param, caughtEx);
+  }
+
+}

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_addjar.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_addjar.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_addjar.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_addjar.q Tue Apr  8 17:38:20 2014
@@ -1,3 +1,7 @@
 set hive.security.authorization.enabled=true;
 set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+
+-- running a sql query to initialize the authorization - not needed in real HS2 mode
+show tables;
+
 add jar ${system:maven.local.repository}/org/apache/hive/hcatalog/hive-hcatalog-core/${system:hive.version}/hive-hcatalog-core-${system:hive.version}.jar;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_addpartition.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_addpartition.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_addpartition.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_addpartition.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 set user.name=user1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 set user.name=user1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q Tue Apr  8 17:38:20 2014
@@ -1,5 +1,5 @@
 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.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set user.name=hive_admin_user;
 set role ADMIN;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q Tue Apr  8 17:38:20 2014
@@ -1,5 +1,5 @@
 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.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set user.name=hive_admin_user;
 set role ADMIN;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q Tue Apr  8 17:38:20 2014
@@ -1,5 +1,5 @@
 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.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set user.name=hive_admin_user;
 set role ADMIN;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q Tue Apr  8 17:38:20 2014
@@ -1,5 +1,5 @@
 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.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set user.name=hive_admin_user;
 set role ADMIN;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_func1.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_func1.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_func1.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_func1.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 set user.name=hive_test_user;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_func2.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_func2.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_func2.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_func2.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 set user.name=hive_test_user;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_macro1.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_macro1.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_macro1.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_macro1.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 set user.name=hive_test_user;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q Tue Apr  8 17:38:20 2014
@@ -1,3 +1,3 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 -- this test will fail because hive_test_user is not in admin role.
 create role r1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_createview.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_createview.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_createview.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_createview.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_ctas.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_ctas.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_ctas.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_ctas.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 set user.name=user1;

Added: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_dfs.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_dfs.q?rev=1585791&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_dfs.q (added)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_dfs.q Tue Apr  8 17:38:20 2014
@@ -0,0 +1,7 @@
+set hive.security.authorization.enabled=true;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+
+-- running a sql query to initialize the authorization - not needed in real HS2 mode
+show tables;
+dfs -ls ${system:test.tmp.dir}/
+

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_disallow_transform.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_disallow_transform.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_disallow_transform.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_disallow_transform.q Tue Apr  8 17:38:20 2014
@@ -1,3 +1,3 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set role ALL;
 SELECT TRANSFORM (*) USING 'cat' AS (key, value) FROM src;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q Tue Apr  8 17:38:20 2014
@@ -1,5 +1,5 @@
 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.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 set user.name=user1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q Tue Apr  8 17:38:20 2014
@@ -1,5 +1,5 @@
 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.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 set user.name=user1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q Tue Apr  8 17:38:20 2014
@@ -1,5 +1,5 @@
 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.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set user.name=hive_admin_user;
 set role ADMIN;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_droppartition.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_droppartition.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_droppartition.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_droppartition.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 
 set user.name=user1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 
 set user.name=user1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 
 set user.name=user1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 
 set user.name=user1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 
 create table if not exists authorization_invalid_v2 (key int, value string);
 grant index on table authorization_invalid_v2 to user hive_test_user;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 set user.name=user1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 set user.name=user1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 set user.name=user1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set hive.security.authorization.enabled=true;
 set user.name=user1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q Tue Apr  8 17:38:20 2014
@@ -1,5 +1,5 @@
 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.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set user.name=hive_admin_user;
 set role ADMIN;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 
 set user.name=user1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q Tue Apr  8 17:38:20 2014
@@ -1,4 +1,4 @@
-set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 
 set user.name=user1;

Modified: hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_role_cycles1.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_role_cycles1.q?rev=1585791&r1=1585790&r2=1585791&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_role_cycles1.q (original)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientnegative/authorization_role_cycles1.q Tue Apr  8 17:38:20 2014
@@ -1,5 +1,5 @@
 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.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
 set user.name=hive_admin_user;
 set role ADMIN;