You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kh...@apache.org on 2015/10/26 19:44:06 UTC

hive git commit: HIVE-9013 : Hive set command exposes metastore db password (Binglin Chang, reviewed by Thejas Nair, Sushanth Sowmyan)

Repository: hive
Updated Branches:
  refs/heads/master 8f9644658 -> a91e1471c


HIVE-9013 : Hive set command exposes metastore db password (Binglin Chang, reviewed by Thejas Nair, Sushanth Sowmyan)


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

Branch: refs/heads/master
Commit: a91e1471cfdd8a8da9bfafc1cb60efd16afbe847
Parents: 8f96446
Author: Sushanth Sowmyan <kh...@gmail.com>
Authored: Mon Oct 26 11:43:02 2015 -0700
Committer: Sushanth Sowmyan <kh...@gmail.com>
Committed: Mon Oct 26 11:43:54 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   | 33 ++++++++++++
 .../apache/hadoop/hive/conf/TestHiveConf.java   | 25 +++++++++
 .../apache/hadoop/hive/ql/exec/Utilities.java   | 13 -----
 .../hadoop/hive/ql/exec/mr/ExecDriver.java      |  2 +-
 .../hadoop/hive/ql/exec/tez/DagUtils.java       |  2 +-
 .../hive/ql/exec/tez/TezSessionState.java       |  2 +-
 .../hadoop/hive/ql/processors/SetProcessor.java | 12 ++++-
 .../hive/ql/processors/TestSetProcessor.java    | 54 ++++++++++++++++++++
 8 files changed, 125 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a91e1471/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index dc79415..643326a 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -45,11 +45,13 @@ import java.io.PrintStream;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -75,6 +77,7 @@ public class HiveConf extends Configuration {
   private static final Map<String, ConfVars> vars = new HashMap<String, ConfVars>();
   private static final Map<String, ConfVars> metaConfs = new HashMap<String, ConfVars>();
   private final List<String> restrictList = new ArrayList<String>();
+  private final Set<String> hiddenSet = new HashSet<String>();
 
   private Pattern modWhiteListPattern = null;
   private volatile boolean isSparkConfigUpdated = false;
@@ -2095,6 +2098,9 @@ public class HiveConf extends Configuration {
     HIVE_CONF_RESTRICTED_LIST("hive.conf.restricted.list",
         "hive.security.authenticator.manager,hive.security.authorization.manager,hive.users.in.admin.role",
         "Comma separated list of configuration options which are immutable at runtime"),
+    HIVE_CONF_HIDDEN_LIST("hive.conf.hidden.list",
+        METASTOREPWD.varname + "," + HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname,
+        "Comma separated list of configuration options which should not be read by normal user like passwords"),
 
     // If this is set all move tasks at the end of a multi-insert query will only begin once all
     // outputs are ready
@@ -2630,6 +2636,10 @@ public class HiveConf extends Configuration {
     }
   }
 
+  public boolean isHiddenConfig(String name) {
+    return hiddenSet.contains(name);
+  }
+
   /**
    * check whether spark related property is updated, which includes spark configurations,
    * RSC configurations and yarn configuration in Spark on YARN mode.
@@ -2979,6 +2989,7 @@ public class HiveConf extends Configuration {
 
     // setup list of conf vars that are not allowed to change runtime
     setupRestrictList();
+    setupHiddenSet();
 
   }
 
@@ -3298,6 +3309,28 @@ public class HiveConf extends Configuration {
     }
     restrictList.add(ConfVars.HIVE_IN_TEST.varname);
     restrictList.add(ConfVars.HIVE_CONF_RESTRICTED_LIST.varname);
+    restrictList.add(ConfVars.HIVE_CONF_HIDDEN_LIST.varname);
+  }
+
+  private void setupHiddenSet() {
+    String hiddenListStr = this.getVar(ConfVars.HIVE_CONF_HIDDEN_LIST);
+    hiddenSet.clear();
+    if (hiddenListStr != null) {
+      for (String entry : hiddenListStr.split(",")) {
+        hiddenSet.add(entry.trim());
+      }
+    }
+  }
+
+  /**
+   * Strips hidden config entries from configuration
+   */
+  public void stripHiddenConfigurations(Configuration conf) {
+    for (String name : hiddenSet) {
+      if (conf.get(name) != null) {
+        conf.set(name, "");
+      }
+    }
   }
 
   public static boolean isLoadMetastoreConfig() {

http://git-wip-us.apache.org/repos/asf/hive/blob/a91e1471/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java b/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java
index e9bde21..3b7a525 100644
--- a/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java
+++ b/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.conf;
 
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.util.Shell;
 import org.apache.hive.common.util.HiveTestUtils;
@@ -117,4 +118,28 @@ public class TestHiveConf {
     Assert.assertEquals(TimeUnit.NANOSECONDS, HiveConf.unitFor("ns", null));
     Assert.assertEquals(TimeUnit.NANOSECONDS, HiveConf.unitFor("nsecs", null));
   }
+
+  @Test
+  public void testHiddenConfig() throws Exception {
+    HiveConf conf = new HiveConf();
+    // check password configs are hidden
+    Assert.assertTrue(conf.isHiddenConfig(HiveConf.ConfVars.METASTOREPWD.varname));
+    Assert.assertTrue(conf.isHiddenConfig(
+        HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname));
+    // check change hidden list should fail
+    try {
+      final String name = HiveConf.ConfVars.HIVE_CONF_HIDDEN_LIST.varname;
+      conf.verifyAndSet(name, "");
+      Assert.fail("Setting config property " + name + " should fail");
+    } catch (IllegalArgumentException e) {
+      // the verifyAndSet in this case is expected to fail with the IllegalArgumentException
+    }
+    // check stripHiddenConfigurations
+    Configuration conf2 = new Configuration(conf);
+    conf2.set(HiveConf.ConfVars.METASTOREPWD.varname, "password");
+    conf2.set(HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname, "password");
+    conf.stripHiddenConfigurations(conf2);
+    Assert.assertEquals("", conf2.get(HiveConf.ConfVars.METASTOREPWD.varname));
+    Assert.assertEquals("", conf2.get(HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/a91e1471/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
index eb2ab56..0eb5f6d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
@@ -3909,19 +3909,6 @@ public final class Utilities {
   }
 
   /**
-   * Strips Hive password details from configuration
-   */
-  public static void stripHivePasswordDetails(Configuration conf) {
-    // Strip out all Hive related password information from the JobConf
-    if (HiveConf.getVar(conf, HiveConf.ConfVars.METASTOREPWD) != null) {
-      HiveConf.setVar(conf, HiveConf.ConfVars.METASTOREPWD, "");
-    }
-    if (HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD) != null) {
-      HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD, "");
-    }
-  }
-
-  /**
    * Returns the full path to the Jar containing the class. It always return a JAR.
    *
    * @param klass

http://git-wip-us.apache.org/repos/asf/hive/blob/a91e1471/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
index b799a17..bed7d63 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
@@ -167,7 +167,7 @@ public class ExecDriver extends Task<MapredWork> implements Serializable, Hadoop
     if (StringUtils.isNotBlank(addedArchives)) {
       HiveConf.setVar(job, ConfVars.HIVEADDEDARCHIVES, addedArchives);
     }
-    Utilities.stripHivePasswordDetails(job);
+    conf.stripHiddenConfigurations(job);
     this.jobExecHelper = new HadoopJobExecHelper(job, console, this, this);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a91e1471/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
index d250b82..46050e8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
@@ -1041,7 +1041,7 @@ public class DagUtils {
     // Removing job credential entry/ cannot be set on the tasks
     conf.unset("mapreduce.job.credentials.binary");
 
-    Utilities.stripHivePasswordDetails(conf);
+    hiveConf.stripHiddenConfigurations(conf);
     return conf;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a91e1471/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java
index 48231be..58be1dc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java
@@ -255,7 +255,7 @@ public class TezSessionState {
 
     // set up the staging directory to use
     tezConfig.set(TezConfiguration.TEZ_AM_STAGING_DIR, tezScratchDir.toUri().toString());
-    Utilities.stripHivePasswordDetails(tezConfig);
+    conf.stripHiddenConfigurations(tezConfig);
 
     ServicePluginsDescriptor servicePluginsDescriptor;
     UserPayload servicePluginPayload = TezUtils.createUserPayloadFromConf(tezConfig);

http://git-wip-us.apache.org/repos/asf/hive/blob/a91e1471/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
index 2226300..9389759 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
@@ -64,6 +64,9 @@ public class SetProcessor implements CommandProcessor {
     for (Object one : p.keySet()) {
       String oneProp = (String) one;
       String oneValue = p.getProperty(oneProp);
+      if (ss.getConf().isHiddenConfig(oneProp)) {
+        continue;
+      }
       sortedMap.put(oneProp, oneValue);
     }
 
@@ -90,7 +93,9 @@ public class SetProcessor implements CommandProcessor {
   private void dumpOption(String s) {
     SessionState ss = SessionState.get();
 
-    if (ss.getConf().get(s) != null) {
+    if (ss.getConf().isHiddenConfig(s)) {
+      ss.out.println(s + " is a hidden config");
+    } else if (ss.getConf().get(s) != null) {
       ss.out.println(s + "=" + ss.getConf().get(s));
     } else if (ss.getHiveVariables().containsKey(s)) {
       ss.out.println(s + "=" + ss.getHiveVariables().get(s));
@@ -241,7 +246,10 @@ public class SetProcessor implements CommandProcessor {
       }
     } else if (varname.indexOf(HIVECONF_PREFIX) == 0) {
       String var = varname.substring(HIVECONF_PREFIX.length());
-      if (ss.getConf().get(var) != null) {
+      if (ss.getConf().isHiddenConfig(var)) {
+        ss.out.println(HIVECONF_PREFIX + var + " is a hidden config");
+        return createProcessorSuccessResponse();
+      } if (ss.getConf().get(var) != null) {
         ss.out.println(HIVECONF_PREFIX + var + "=" + ss.getConf().get(var));
         return createProcessorSuccessResponse();
       } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/a91e1471/ql/src/test/org/apache/hadoop/hive/ql/processors/TestSetProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/processors/TestSetProcessor.java b/ql/src/test/org/apache/hadoop/hive/ql/processors/TestSetProcessor.java
new file mode 100644
index 0000000..bff643a
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/processors/TestSetProcessor.java
@@ -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.processors;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.Test;
+
+public class TestSetProcessor {
+
+  @Test
+  public void testHiddenConfig() throws Exception {
+    HiveConf conf = new HiveConf();
+    SessionState.start(conf);
+    SessionState state = SessionState.get();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    state.out = new PrintStream(baos);
+    SetProcessor processor = new SetProcessor();
+    processor.run("");
+    state.out.flush();
+    String ret = baos.toString();
+    Assert.assertFalse(ret.contains(HiveConf.ConfVars.METASTOREPWD.varname + "="));
+    Assert.assertFalse(ret.contains(HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname + "="));
+
+    baos = new ByteArrayOutputStream();
+    state.out = new PrintStream(baos);
+    processor.run(HiveConf.ConfVars.METASTOREPWD.varname);
+    state.out.flush();
+    ret = new String(baos.toByteArray());
+    Assert.assertTrue(baos.toString().contains("hidden"));
+  }
+
+}