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/10/23 20:09:04 UTC

svn commit: r1633906 - in /hive/branches/branch-0.14: common/src/java/org/apache/hadoop/hive/conf/ itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/ ql/src/java/org/apache/hadoop/hive/ql/security/authorization/pl...

Author: thejas
Date: Thu Oct 23 18:09:04 2014
New Revision: 1633906

URL: http://svn.apache.org/r1633906
Log:
HIVE-8534 : sql std auth : update configuration whitelist for 0.14 (Thejas Nair, reviewed by Gunther Hagleitner, Lefty Leverenz)

Added:
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/SettableConfigUpdater.java
    hive/branches/branch-0.14/ql/src/test/queries/clientnegative/authorization_set_invalidconf.q
    hive/branches/branch-0.14/ql/src/test/results/clientnegative/authorization_set_invalidconf.q.out
Modified:
    hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/branches/branch-0.14/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessControllerForTest.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessControllerWrapper.java
    hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java
    hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerHS2.java
    hive/branches/branch-0.14/ql/src/test/queries/clientnegative/authorization_disallow_transform.q
    hive/branches/branch-0.14/ql/src/test/results/clientnegative/authorization_disallow_transform.q.out

Modified: hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1633906&r1=1633905&r2=1633906&view=diff
==============================================================================
--- hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Thu Oct 23 18:09:04 2014
@@ -53,6 +53,8 @@ import org.apache.hadoop.security.UserGr
 import org.apache.hadoop.util.Shell;
 import org.apache.hive.common.HiveCompat;
 
+import com.google.common.base.Joiner;
+
 /**
  * Hive Configuration.
  */
@@ -75,9 +77,7 @@ public class HiveConf extends Configurat
   private static final Map<String, ConfVars> metaConfs = new HashMap<String, ConfVars>();
   private final List<String> restrictList = new ArrayList<String>();
 
-  private boolean isWhiteListRestrictionEnabled = false;
-  private final List<String> modWhiteList = new ArrayList<String>();
-
+  private Pattern modWhiteListPattern = null;
 
   static {
     ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
@@ -1402,11 +1402,23 @@ public class HiveConf extends Configurat
         "the privileges automatically granted to the owner whenever a table gets created.\n" +
         "An example like \"select,drop\" will grant select and drop privilege to the owner of the table"),
 
-    // if this is not set default value is added by sql standard authorizer.
+    // if this is not set default value is set during config initialization
     // 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", "",
-        "interal variable. List of modifiable configurations by user."),
+    HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST(
+        "hive.security.authorization.sqlstd.confwhitelist", "",
+        "List of comma separated Java regexes. Configurations parameters that match these\n" +
+        "regexes can be modified by user when SQL standard authorization is enabled.\n" +
+        "To get the default value, use the 'set <param>' command.\n" +
+        "Note that the hive.conf.restricted.list checks are still enforced after the white list\n" +
+        "check"),
+
+    HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST_APPEND(
+        "hive.security.authorization.sqlstd.confwhitelist.append", "",
+        "List of comma separated Java regexes, to be appended to list set in\n" +
+        "hive.security.authorization.sqlstd.confwhitelist. Using this list instead\n" +
+        "of updating the original list means that you can append to the defaults\n" +
+        "set by SQL standard authorization instead of replacing it entirely."),
 
     HIVE_CLI_PRINT_HEADER("hive.cli.print.header", false, "Whether to print the names of the columns in query output."),
 
@@ -2031,8 +2043,9 @@ public class HiveConf extends Configurat
   }
 
   public void verifyAndSet(String name, String value) throws IllegalArgumentException {
-    if (isWhiteListRestrictionEnabled) {
-      if (!modWhiteList.contains(name)) {
+    if (modWhiteListPattern != null) {
+      Matcher wlMatcher = modWhiteListPattern.matcher(name);
+      if (!wlMatcher.matches()) {
         throw new IllegalArgumentException("Cannot modify " + name + " at runtime. "
             + "It is not in list of params that are allowed to be modified at runtime");
       }
@@ -2369,11 +2382,146 @@ public class HiveConf extends Configurat
         unset(key);
       }
     }
+
+    setupSQLStdAuthWhiteList();
+
     // setup list of conf vars that are not allowed to change runtime
     setupRestrictList();
+
+  }
+
+  /**
+   * If the config whitelist param for sql standard authorization is not set, set it up here.
+   */
+  private void setupSQLStdAuthWhiteList() {
+    String whiteListParamsStr = getVar(ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST);
+    if (whiteListParamsStr == null || whiteListParamsStr.trim().isEmpty()) {
+      // set the default configs in whitelist
+      whiteListParamsStr = getSQLStdAuthDefaultWhiteListPattern();
+    }
+    setVar(ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST, whiteListParamsStr);
+  }
+
+  private static String getSQLStdAuthDefaultWhiteListPattern() {
+    // create the default white list from list of safe config params
+    // and regex list
+    String confVarPatternStr = Joiner.on("|").join(convertVarsToRegex(sqlStdAuthSafeVarNames));
+    String regexPatternStr = Joiner.on("|").join(sqlStdAuthSafeVarNameRegexes);
+    return regexPatternStr + "|" + confVarPatternStr;
+  }
+
+  /**
+   * @param paramList  list of parameter strings
+   * @return list of parameter strings with "." replaced by "\."
+   */
+  private static String[] convertVarsToRegex(String[] paramList) {
+    String[] regexes = new String[paramList.length];
+    for(int i=0; i<paramList.length; i++) {
+      regexes[i] = paramList[i].replace(".", "\\." );
+    }
+    return regexes;
   }
 
   /**
+   * Default list of modifiable config parameters for sql standard authorization
+   * For internal use only.
+   */
+  private static final String [] sqlStdAuthSafeVarNames = new String [] {
+    ConfVars.BYTESPERREDUCER.varname,
+    ConfVars.CLIENT_STATS_COUNTERS.varname,
+    ConfVars.DEFAULTPARTITIONNAME.varname,
+    ConfVars.DROPIGNORESNONEXISTENT.varname,
+    ConfVars.HIVECOUNTERGROUP.varname,
+    ConfVars.HIVEENFORCEBUCKETING.varname,
+    ConfVars.HIVEENFORCEBUCKETMAPJOIN.varname,
+    ConfVars.HIVEENFORCESORTING.varname,
+    ConfVars.HIVEENFORCESORTMERGEBUCKETMAPJOIN.varname,
+    ConfVars.HIVEEXPREVALUATIONCACHE.varname,
+    ConfVars.HIVEGROUPBYSKEW.varname,
+    ConfVars.HIVEHASHTABLELOADFACTOR.varname,
+    ConfVars.HIVEHASHTABLETHRESHOLD.varname,
+    ConfVars.HIVEIGNOREMAPJOINHINT.varname,
+    ConfVars.HIVELIMITMAXROWSIZE.varname,
+    ConfVars.HIVEMAPREDMODE.varname,
+    ConfVars.HIVEMAPSIDEAGGREGATE.varname,
+    ConfVars.HIVEOPTIMIZEMETADATAQUERIES.varname,
+    ConfVars.HIVEROWOFFSET.varname,
+    ConfVars.HIVEVARIABLESUBSTITUTE.varname,
+    ConfVars.HIVEVARIABLESUBSTITUTEDEPTH.varname,
+    ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_INCLUDEFUNCNAME.varname,
+    ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_LABEL.varname,
+    ConfVars.HIVE_CHECK_CROSS_PRODUCT.varname,
+    ConfVars.HIVE_COMPAT.varname,
+    ConfVars.HIVE_CONCATENATE_CHECK_INDEX.varname,
+    ConfVars.HIVE_DISPLAY_PARTITION_COLUMNS_SEPARATELY.varname,
+    ConfVars.HIVE_ERROR_ON_EMPTY_PARTITION.varname,
+    ConfVars.HIVE_EXECUTION_ENGINE.varname,
+    ConfVars.HIVE_EXIM_URI_SCHEME_WL.varname,
+    ConfVars.HIVE_FILE_MAX_FOOTER.varname,
+    ConfVars.HIVE_HADOOP_SUPPORTS_SUBDIRECTORIES.varname,
+    ConfVars.HIVE_INSERT_INTO_MULTILEVEL_DIRS.varname,
+    ConfVars.HIVE_LOCALIZE_RESOURCE_NUM_WAIT_ATTEMPTS.varname,
+    ConfVars.HIVE_MULTI_INSERT_MOVE_TASKS_SHARE_DEPENDENCIES.varname,
+    ConfVars.HIVE_QUOTEDID_SUPPORT.varname,
+    ConfVars.HIVE_RESULTSET_USE_UNIQUE_COLUMN_NAMES.varname,
+    ConfVars.HIVE_STATS_COLLECT_PART_LEVEL_STATS.varname,
+    ConfVars.JOB_DEBUG_CAPTURE_STACKTRACES.varname,
+    ConfVars.JOB_DEBUG_TIMEOUT.varname,
+    ConfVars.MAXCREATEDFILES.varname,
+    ConfVars.MAXREDUCERS.varname,
+    ConfVars.OUTPUT_FILE_EXTENSION.varname,
+    ConfVars.SHOW_JOB_FAIL_DEBUG_INFO.varname,
+    ConfVars.TASKLOG_DEBUG_TIMEOUT.varname,
+  };
+
+  /**
+   * Default list of regexes for config parameters that are modifiable with
+   * sql standard authorization enabled
+   */
+  static final String [] sqlStdAuthSafeVarNameRegexes = new String [] {
+    "hive\\.auto\\..*",
+    "hive\\.cbo\\..*",
+    "hive\\.convert\\..*",
+    "hive\\.exec\\..*\\.dynamic\\.partitions\\..*",
+    "hive\\.exec\\.compress\\..*",
+    "hive\\.exec\\.infer\\..*",
+    "hive\\.exec\\.mode.local\\..*",
+    "hive\\.exec\\.orc\\..*",
+    "hive\\.fetch.task\\..*",
+    "hive\\.hbase\\..*",
+    "hive\\.index\\..*",
+    "hive\\.index\\..*",
+    "hive\\.intermediate\\..*",
+    "hive\\.join\\..*",
+    "hive\\.limit\\..*",
+    "hive\\.mapjoin\\..*",
+    "hive\\.merge\\..*",
+    "hive\\.optimize\\..*",
+    "hive\\.orc\\..*",
+    "hive\\.outerjoin\\..*",
+    "hive\\.ppd\\..*",
+    "hive\\.prewarm\\..*",
+    "hive\\.skewjoin\\..*",
+    "hive\\.smbjoin\\..*",
+    "hive\\.stats\\..*",
+    "hive\\.tez\\..*",
+    "hive\\.vectorized\\..*",
+    "mapred\\.map\\..*",
+    "mapred\\.reduce\\..*",
+    "mapred\\.output\\.compression\\.codec",
+    "mapreduce\\.job\\.reduce\\.slowstart\\.completedmaps",
+    "mapreduce\\.job\\.queuename",
+    "mapreduce\\.input\\.fileinputformat\\.split\\.minsize",
+    "mapreduce\\.map\\..*",
+    "mapreduce\\.reduce\\..*",
+    "tez\\.am\\..*",
+    "tez\\.task\\..*",
+    "tez\\.runtime\\..*",
+  };
+
+
+
+  /**
    * Apply system properties to this object if the property name is defined in ConfVars
    * and the value is non-null and not an empty string.
    */
@@ -2515,26 +2663,16 @@ 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
+   * Set white list of parameters that are allowed to be modified
    *
-   * @param paramname
+   * @param paramNameRegex
    */
   @LimitedPrivate(value = { "Currently only for use by HiveAuthorizer" })
-  public void addToModifiableWhiteList(String paramname) {
-    if (paramname == null) {
+  public void setModifiableWhiteListRegex(String paramNameRegex) {
+    if (paramNameRegex == null) {
       return;
     }
-    modWhiteList.add(paramname);
+    modWhiteListPattern = Pattern.compile(paramNameRegex);
   }
 
   /**

Modified: hive/branches/branch-0.14/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.14/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessControllerForTest.java?rev=1633906&r1=1633905&r2=1633906&view=diff
==============================================================================
--- hive/branches/branch-0.14/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessControllerForTest.java (original)
+++ hive/branches/branch-0.14/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessControllerForTest.java Thu Oct 23 18:09:04 2014
@@ -39,12 +39,11 @@ public class SQLStdHiveAccessControllerF
 
 
   @Override
-  public void applyAuthorizationConfigPolicy(HiveConf hiveConf) {
+  public void applyAuthorizationConfigPolicy(HiveConf hiveConf) throws HiveAuthzPluginException {
     super.applyAuthorizationConfigPolicy(hiveConf);
 
     // remove restrictions on the variables that can be set using set command
-    hiveConf.setIsModWhiteListEnabled(false);
-
+    hiveConf.setModifiableWhiteListRegex(".*");
   }
 
 }

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java?rev=1633906&r1=1633905&r2=1633906&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java Thu Oct 23 18:09:04 2014
@@ -70,5 +70,5 @@ public interface HiveAccessController {
   List<HiveRoleGrant> getRoleGrantInfoForPrincipal(HivePrincipal principal) throws HiveAuthzPluginException,
       HiveAccessControlException;
 
-  void applyAuthorizationConfigPolicy(HiveConf hiveConf);
+  void applyAuthorizationConfigPolicy(HiveConf hiveConf) throws HiveAuthzPluginException;
 }

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java?rev=1633906&r1=1633905&r2=1633906&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java Thu Oct 23 18:09:04 2014
@@ -191,8 +191,9 @@ public interface HiveAuthorizer {
    * Modify the given HiveConf object to configure authorization related parameters
    * or other parameters related to hive security
    * @param hiveConf
+   * @throws HiveAuthzPluginException
    */
-  public void applyAuthorizationConfigPolicy(HiveConf hiveConf);
+  public void applyAuthorizationConfigPolicy(HiveConf hiveConf) throws HiveAuthzPluginException;
 
 }
 

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java?rev=1633906&r1=1633905&r2=1633906&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java Thu Oct 23 18:09:04 2014
@@ -124,7 +124,7 @@ public class HiveAuthorizerImpl implemen
   }
 
   @Override
-  public void applyAuthorizationConfigPolicy(HiveConf hiveConf) {
+  public void applyAuthorizationConfigPolicy(HiveConf hiveConf) throws HiveAuthzPluginException {
     accessController.applyAuthorizationConfigPolicy(hiveConf);
   }
 }

Added: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/SettableConfigUpdater.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/SettableConfigUpdater.java?rev=1633906&view=auto
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/SettableConfigUpdater.java (added)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/SettableConfigUpdater.java Thu Oct 23 18:09:04 2014
@@ -0,0 +1,63 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.hive.common.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.hive.common.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+
+/**
+ * Helper class that can be used by authorization implementations to set a
+ * default list of 'safe' HiveConf parameters that can be edited by user. It
+ * uses HiveConf white list parameters to enforce this. This can be called from
+ * HiveAuthorizer.applyAuthorizationConfigPolicy
+ *
+ * The set of config parameters that can be set is restricted to parameters that
+ * don't allow for any code injection, and config parameters that are not
+ * considered an 'admin config' option.
+ *
+ */
+@LimitedPrivate(value = { "Apache Argus (incubating)" })
+@Evolving
+@Unstable
+public class SettableConfigUpdater {
+
+  public static void setHiveConfWhiteList(HiveConf hiveConf) throws HiveAuthzPluginException {
+
+    String whiteListParamsStr = hiveConf
+        .getVar(ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST);
+
+    if(whiteListParamsStr == null && whiteListParamsStr.trim().isEmpty()) {
+      throw new HiveAuthzPluginException("Configuration parameter "
+          + ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST.varname
+          + " is not iniatialized.");
+    }
+
+    // append regexes that user wanted to add
+    String whiteListAppend = hiveConf
+        .getVar(ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST_APPEND);
+    if (whiteListAppend != null && !whiteListAppend.trim().equals("")) {
+      whiteListParamsStr = whiteListParamsStr + "|" + whiteListAppend;
+    }
+
+    hiveConf.setModifiableWhiteListRegex(whiteListParamsStr);
+  }
+
+}

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java?rev=1633906&r1=1633905&r2=1633906&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java Thu Oct 23 18:09:04 2014
@@ -60,9 +60,9 @@ import org.apache.hadoop.hive.ql.securit
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveRoleGrant;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.SettableConfigUpdater;
 import org.apache.thrift.TException;
 
-import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableSet;
 
 /**
@@ -610,72 +610,8 @@ 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) {
+  public void applyAuthorizationConfigPolicy(HiveConf hiveConf) throws HiveAuthzPluginException {
     // First apply configuration applicable to both Hive Cli and HiveServer2
     // Not adding any authorization related restrictions to hive cli
     // grant all privileges for table to its owner - set this in cli as well so that owner
@@ -683,28 +619,21 @@ public class SQLStdHiveAccessController 
     hiveConf.setVar(ConfVars.HIVE_AUTHORIZATION_TABLE_OWNER_GRANTS, "INSERT,SELECT,UPDATE,DELETE");
 
     // Apply rest of the configuration only to HiveServer2
-    if(sessionCtx.getClientType() == CLIENT_TYPE.HIVESERVER2) {
+    if (sessionCtx.getClientType() == CLIENT_TYPE.HIVESERVER2
+        && hiveConf.getBoolVar(ConfVars.HIVE_AUTHORIZATION_ENABLED)) {
+
       // 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();
+        hooks = hooks + "," + DisallowTransformHook.class.getName();
       }
       LOG.debug("Configuring hooks : " + hooks);
       hiveConf.setVar(ConfVars.PREEXECHOOKS, hooks);
 
-      // 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);
-      }
+      SettableConfigUpdater.setHiveConfWhiteList(hiveConf);
+
     }
   }
 

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessControllerWrapper.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessControllerWrapper.java?rev=1633906&r1=1633905&r2=1633906&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessControllerWrapper.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessControllerWrapper.java Thu Oct 23 18:09:04 2014
@@ -174,7 +174,7 @@ public class SQLStdHiveAccessControllerW
   }
 
   @Override
-  public void applyAuthorizationConfigPolicy(HiveConf hiveConf) {
+  public void applyAuthorizationConfigPolicy(HiveConf hiveConf) throws HiveAuthzPluginException {
     hiveAccessController.applyAuthorizationConfigPolicy(hiveConf);
   }
 

Modified: hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java?rev=1633906&r1=1633905&r2=1633906&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java (original)
+++ hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java Thu Oct 23 18:09:04 2014
@@ -55,13 +55,10 @@ public class TestSQLStdHiveAccessControl
     assertFalse("Check for transform query disabling hook",
         processedConf.getVar(ConfVars.PREEXECHOOKS).contains(DisallowTransformHook.class.getName()));
 
-    // check that set param whitelist is not set
-    assertTrue(processedConf.getVar(ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST) == null
-        || processedConf.getVar(ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST).trim()
-            .equals(""));
-
     // verify that some dummy param can be set
     processedConf.verifyAndSet("dummy.param", "dummy.val");
+    processedConf.verifyAndSet(ConfVars.HIVE_AUTHORIZATION_ENABLED.varname, "true");
+
   }
 
   private HiveAuthzSessionContext getCLISessionCtx() {

Modified: hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerHS2.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerHS2.java?rev=1633906&r1=1633905&r2=1633906&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerHS2.java (original)
+++ hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerHS2.java Thu Oct 23 18:09:04 2014
@@ -20,6 +20,11 @@ package org.apache.hadoop.hive.ql.securi
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.security.HadoopDefaultAuthenticator;
@@ -42,20 +47,53 @@ public class TestSQLStdHiveAccessControl
    * policy on hiveconf correctly
    *
    * @throws HiveAuthzPluginException
+   * @throws IllegalAccessException
+   * @throws NoSuchFieldException
+   * @throws IllegalArgumentException
+   * @throws SecurityException
    */
   @Test
-  public void testConfigProcessing() throws HiveAuthzPluginException {
-    HiveConf processedConf = new HiveConf();
+  public void testConfigProcessing() throws HiveAuthzPluginException, SecurityException,
+      IllegalArgumentException, NoSuchFieldException, IllegalAccessException {
+    HiveConf processedConf = newAuthEnabledConf();
     SQLStdHiveAccessController accessController = new SQLStdHiveAccessController(null,
-        processedConf, new HadoopDefaultAuthenticator(), getHS2SessionCtx()
-        );
+        processedConf, new HadoopDefaultAuthenticator(), getHS2SessionCtx());
     accessController.applyAuthorizationConfigPolicy(processedConf);
 
     // 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);
+    List<String> settableParams = getSettableParams();
+    verifyParamSettability(settableParams, processedConf);
+
+  }
+
+  private HiveConf newAuthEnabledConf() {
+    HiveConf conf = new HiveConf();
+    conf.setBoolVar(ConfVars.HIVE_AUTHORIZATION_ENABLED, true);
+    return conf;
+  }
+
+  /**
+   * @return list of parameters that should be possible to set
+   */
+  private List<String> getSettableParams() throws SecurityException, NoSuchFieldException,
+      IllegalArgumentException, IllegalAccessException {
+    // get all the variable names being converted to regex in HiveConf, using reflection
+    Field varNameField = HiveConf.class.getDeclaredField("sqlStdAuthSafeVarNames");
+    varNameField.setAccessible(true);
+    List<String> confVarList = Arrays.asList((String[]) varNameField.get(null));
+
+    // create list with variables that match some of the regexes
+    List<String> confVarRegexList = Arrays.asList("hive.convert.join.bucket.mapjoin.tez",
+        "hive.optimize.index.filter.compact.maxsize", "hive.tez.dummy", "tez.task.dummy");
+
+    // combine two lists
+    List<String> varList = new ArrayList<String>();
+    varList.addAll(confVarList);
+    varList.addAll(confVarRegexList);
+    return varList;
 
   }
 
@@ -70,7 +108,7 @@ public class TestSQLStdHiveAccessControl
    * @param settableParams
    * @param processedConf
    */
-  private void verifyParamSettability(String [] settableParams, HiveConf processedConf) {
+  private void verifyParamSettability(List<String> settableParams, HiveConf processedConf) {
     // verify that the whitlelist params can be set
     for (String param : settableParams) {
       try {
@@ -90,24 +128,42 @@ public class TestSQLStdHiveAccessControl
   }
 
   /**
-   * Test that modifying HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST config works
+   * Test that setting HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST_APPEND config works
+   * @throws HiveAuthzPluginException
+   */
+  @Test
+  public void testConfigProcessingCustomSetWhitelistAppend() throws HiveAuthzPluginException {
+    // append new config params to whitelist
+    List<String> paramRegexes = Arrays.asList("hive.ctest.param", "hive.abc..*");
+    List<String> settableParams = Arrays.asList("hive.ctest.param", "hive.abc.def");
+    verifySettability(paramRegexes, settableParams,
+        ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST_APPEND);
+  }
+
+  /**
+   * Test that setting HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST config works
    * @throws HiveAuthzPluginException
    */
   @Test
   public void testConfigProcessingCustomSetWhitelist() throws HiveAuthzPluginException {
+    // append new config params to whitelist
+    List<String> paramRegexes = Arrays.asList("hive.ctest.param", "hive.abc..*");
+    List<String> settableParams = Arrays.asList("hive.ctest.param", "hive.abc.def");
+    verifySettability(paramRegexes, settableParams,
+        ConfVars.HIVE_AUTHORIZATION_SQL_STD_AUTH_CONFIG_WHITELIST);
+  }
 
-    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));
+  private void verifySettability(List<String> paramRegexes, List<String> settableParams,
+      ConfVars whiteListParam) throws HiveAuthzPluginException {
+    HiveConf processedConf = newAuthEnabledConf();
+    processedConf.setVar(whiteListParam,
+        Joiner.on("|").join(paramRegexes));
 
     SQLStdHiveAccessController accessController = new SQLStdHiveAccessController(null,
         processedConf, new HadoopDefaultAuthenticator(), getHS2SessionCtx());
     accessController.applyAuthorizationConfigPolicy(processedConf);
-    verifyParamSettability(settableParams, processedConf);
 
+    verifyParamSettability(settableParams, processedConf);
   }
 
   private void assertConfModificationException(HiveConf processedConf, String param) {
@@ -120,4 +176,5 @@ public class TestSQLStdHiveAccessControl
     assertTrue("Exception should be thrown while modifying the param " + param, caughtEx);
   }
 
+
 }

Modified: hive/branches/branch-0.14/ql/src/test/queries/clientnegative/authorization_disallow_transform.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/queries/clientnegative/authorization_disallow_transform.q?rev=1633906&r1=1633905&r2=1633906&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/queries/clientnegative/authorization_disallow_transform.q (original)
+++ hive/branches/branch-0.14/ql/src/test/queries/clientnegative/authorization_disallow_transform.q Thu Oct 23 18:09:04 2014
@@ -1,4 +1,6 @@
 set hive.test.authz.sstd.hs2.mode=true;
 set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authorization.enabled=true;
 set role ALL;
-SELECT TRANSFORM (*) USING 'cat' AS (key, value) FROM src;
+create table t1(i int);
+SELECT TRANSFORM (*) USING 'cat' AS (key, value) FROM t1;

Added: hive/branches/branch-0.14/ql/src/test/queries/clientnegative/authorization_set_invalidconf.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/queries/clientnegative/authorization_set_invalidconf.q?rev=1633906&view=auto
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/queries/clientnegative/authorization_set_invalidconf.q (added)
+++ hive/branches/branch-0.14/ql/src/test/queries/clientnegative/authorization_set_invalidconf.q Thu Oct 23 18:09:04 2014
@@ -0,0 +1,8 @@
+set hive.test.authz.sstd.hs2.mode=true;
+set hive.security.authorization.enabled=true;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+
+-- run a sql query to initialize authorization, then try setting a allowed config and then a disallowed config param
+use default;
+set hive.optimize.listbucketing=true;
+set hive.security.authorization.enabled=true;

Modified: hive/branches/branch-0.14/ql/src/test/results/clientnegative/authorization_disallow_transform.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/results/clientnegative/authorization_disallow_transform.q.out?rev=1633906&r1=1633905&r2=1633906&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/results/clientnegative/authorization_disallow_transform.q.out (original)
+++ hive/branches/branch-0.14/ql/src/test/results/clientnegative/authorization_disallow_transform.q.out Thu Oct 23 18:09:04 2014
@@ -2,9 +2,17 @@ PREHOOK: query: set role ALL
 PREHOOK: type: SHOW_ROLES
 POSTHOOK: query: set role ALL
 POSTHOOK: type: SHOW_ROLES
-PREHOOK: query: SELECT TRANSFORM (*) USING 'cat' AS (key, value) FROM src
+PREHOOK: query: create table t1(i int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t1
+POSTHOOK: query: create table t1(i int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t1
+PREHOOK: query: SELECT TRANSFORM (*) USING 'cat' AS (key, value) FROM t1
 PREHOOK: type: QUERY
-PREHOOK: Input: default@src
+PREHOOK: Input: default@t1
 #### A masked pattern was here ####
 FAILED: Hive Internal Error: org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException(Query with transform clause is disallowed in current configuration.)
 org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException: Query with transform clause is disallowed in current configuration.

Added: hive/branches/branch-0.14/ql/src/test/results/clientnegative/authorization_set_invalidconf.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/test/results/clientnegative/authorization_set_invalidconf.q.out?rev=1633906&view=auto
==============================================================================
--- hive/branches/branch-0.14/ql/src/test/results/clientnegative/authorization_set_invalidconf.q.out (added)
+++ hive/branches/branch-0.14/ql/src/test/results/clientnegative/authorization_set_invalidconf.q.out Thu Oct 23 18:09:04 2014
@@ -0,0 +1,9 @@
+PREHOOK: query: -- run a sql query to initialize authorization, then try setting a allowed config and then a disallowed config param
+use default
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:default
+POSTHOOK: query: -- run a sql query to initialize authorization, then try setting a allowed config and then a disallowed config param
+use default
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:default
+Query returned non-zero code: 1, cause: Cannot modify hive.security.authorization.enabled at runtime. It is not in list of params that are allowed to be modified at runtime