You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ct...@apache.org on 2016/09/02 13:44:55 UTC

hive git commit: HIVE-14426: Extensive logging on info level in WebHCat (Peter Vary via Chaoyu Tang)

Repository: hive
Updated Branches:
  refs/heads/master 131631e0e -> fa1d8c760


HIVE-14426: Extensive logging on info level in WebHCat (Peter Vary via Chaoyu Tang)


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

Branch: refs/heads/master
Commit: fa1d8c7602a12726c179cfc4432382bf22f2b7d1
Parents: 131631e
Author: ctang <ct...@cloudera.com>
Authored: Fri Sep 2 09:43:48 2016 -0400
Committer: ctang <ct...@cloudera.com>
Committed: Fri Sep 2 09:43:48 2016 -0400

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/common/LogUtils.java | 22 +++++++++++
 .../org/apache/hadoop/hive/conf/HiveConf.java   | 20 ++--------
 .../apache/hadoop/hive/conf/HiveConfUtil.java   | 39 +++++++++++++++++++-
 .../apache/hadoop/hive/common/TestLogUtils.java | 34 +++++++++++++++++
 .../hive/hcatalog/templeton/AppConfig.java      | 15 +++++++-
 .../hcatalog/templeton/tool/TempletonUtils.java |  4 +-
 .../apache/hadoop/hive/ql/exec/Utilities.java   | 23 ------------
 .../hadoop/hive/ql/exec/mr/MapredLocalTask.java |  3 +-
 .../ql/exec/spark/HiveSparkClientFactory.java   | 11 +++---
 .../hadoop/hive/ql/exec/TestUtilities.java      | 12 ------
 10 files changed, 121 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/fa1d8c76/common/src/java/org/apache/hadoop/hive/common/LogUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/LogUtils.java b/common/src/java/org/apache/hadoop/hive/common/LogUtils.java
index 599e798..c2a0d9a 100644
--- a/common/src/java/org/apache/hadoop/hive/common/LogUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/LogUtils.java
@@ -42,6 +42,12 @@ public class LogUtils {
   private static final String HIVE_EXEC_L4J = "hive-exec-log4j2.properties";
   private static final Logger l4j = LoggerFactory.getLogger(LogUtils.class);
 
+  /**
+   * Constants for log masking
+   */
+  private static String KEY_TO_MASK_WITH = "password";
+  private static String MASKED_VALUE = "###_MASKED_###";
+
   @SuppressWarnings("serial")
   public static class LogInitializationException extends Exception {
     public LogInitializationException(String msg) {
@@ -171,4 +177,20 @@ public class LogUtils {
         + conf.getHiveSiteLocation().getPath());
     }
   }
+
+  /**
+   * Returns MASKED_VALUE if the key contains KEY_TO_MASK_WITH or the original property otherwise.
+   * Used to mask environment variables, and properties in logs which contain passwords
+   * @param key The property key to check
+   * @param value The original value of the property
+   * @return The masked property value
+   */
+  public static String maskIfPassword(String key, String value) {
+    if (key!=null && value!=null) {
+      if (key.toLowerCase().indexOf(KEY_TO_MASK_WITH) != -1) {
+        return MASKED_VALUE;
+      }
+    }
+    return value;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/fa1d8c76/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 aa03b63..13cfdf1 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3867,8 +3867,8 @@ public class HiveConf extends Configuration {
 
     // setup list of conf vars that are not allowed to change runtime
     setupRestrictList();
-    setupHiddenSet();
-
+    hiddenSet.clear();
+    hiddenSet.addAll(HiveConfUtil.getHiddenSet(this));
   }
 
   /**
@@ -4221,25 +4221,11 @@ public class HiveConf extends Configuration {
     restrictList.add(ConfVars.HIVE_CONF_INTERNAL_VARIABLE_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, "");
-      }
-    }
+    HiveConfUtil.stripConfigurations(conf, hiddenSet);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/fa1d8c76/common/src/java/org/apache/hadoop/hive/conf/HiveConfUtil.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConfUtil.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConfUtil.java
index 073a978..16c2eaf 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConfUtil.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConfUtil.java
@@ -25,9 +25,11 @@ import java.io.File;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.StringTokenizer;
 
 /**
@@ -59,7 +61,42 @@ public class HiveConfUtil {
     dumpConfig(conf, sb);
     return sb.append("END========\"new HiveConf()\"========\n");
   }
-  public static void dumpConfig(Configuration conf, StringBuilder sb) {
+
+  /**
+   * Getting the set of the hidden configurations
+   * @param configuration The original configuration
+   * @return The list of the configuration values to hide
+   */
+  public static Set<String> getHiddenSet(Configuration configuration) {
+    Set<String> hiddenSet = new HashSet<String>();
+    String hiddenListStr = HiveConf.getVar(configuration, HiveConf.ConfVars.HIVE_CONF_HIDDEN_LIST);
+    if (hiddenListStr != null) {
+      for (String entry : hiddenListStr.split(",")) {
+        hiddenSet.add(entry.trim());
+      }
+    }
+    return hiddenSet;
+  }
+
+  /**
+   * Strips hidden config entries from configuration
+   * @param conf The configuration to strip from
+   * @param hiddenSet The values to strip
+   */
+  public static void stripConfigurations(Configuration conf, Set<String> hiddenSet) {
+    for (String name : hiddenSet) {
+      if (conf.get(name) != null) {
+        conf.set(name, "");
+      }
+    }
+  }
+
+  public static void dumpConfig(Configuration originalConf, StringBuilder sb) {
+    Set<String> hiddenSet = getHiddenSet(originalConf);
+    sb.append("Values omitted for security reason if present: ").append(hiddenSet).append("\n");
+    Configuration conf = new Configuration(originalConf);
+    stripConfigurations(conf, hiddenSet);
+
     Iterator<Map.Entry<String, String>> configIter = conf.iterator();
     List<Map.Entry<String, String>> configVals = new ArrayList<>();
     while(configIter.hasNext()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/fa1d8c76/common/src/test/org/apache/hadoop/hive/common/TestLogUtils.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/TestLogUtils.java b/common/src/test/org/apache/hadoop/hive/common/TestLogUtils.java
new file mode 100644
index 0000000..923ac2d
--- /dev/null
+++ b/common/src/test/org/apache/hadoop/hive/common/TestLogUtils.java
@@ -0,0 +1,34 @@
+package org.apache.hadoop.hive.common;
+/**
+ * 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.
+ */
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestLogUtils {
+  @Test
+  public void testMaskIfPassword() {
+    Assert.assertNull(LogUtils.maskIfPassword("",null));
+    Assert.assertNull(LogUtils.maskIfPassword(null,null));
+    Assert.assertEquals("test", LogUtils.maskIfPassword(null,"test"));
+    Assert.assertEquals("test2", LogUtils.maskIfPassword("any","test2"));
+    Assert.assertEquals("###_MASKED_###", LogUtils.maskIfPassword("password","test3"));
+    Assert.assertEquals("###_MASKED_###", LogUtils.maskIfPassword("a_passWord","test4"));
+    Assert.assertEquals("###_MASKED_###", LogUtils.maskIfPassword("password_a","test5"));
+    Assert.assertEquals("###_MASKED_###", LogUtils.maskIfPassword("a_PassWord_a","test6"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/fa1d8c76/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/AppConfig.java
----------------------------------------------------------------------
diff --git a/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/AppConfig.java b/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/AppConfig.java
index dd1208b..54d0907 100644
--- a/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/AppConfig.java
+++ b/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/AppConfig.java
@@ -200,14 +200,19 @@ public class AppConfig extends Configuration {
    * been installed.  We need pass some properties to that client to make sure it connects to the
    * right Metastore, configures Tez, etc.  Here we look for such properties in hive config,
    * and set a comma-separated list of key values in {@link #HIVE_PROPS_NAME}.
+   * The HIVE_CONF_HIDDEN_LIST should be handled separately too - this also should be copied from
+   * the hive config to the webhcat config if not defined there.
    * Note that the user may choose to set the same keys in HIVE_PROPS_NAME directly, in which case
    * those values should take precedence.
    */
   private void handleHiveProperties() {
     HiveConf hiveConf = new HiveConf();//load hive-site.xml from classpath
     List<String> interestingPropNames = Arrays.asList(
-      "hive.metastore.uris","hive.metastore.sasl.enabled",
-      "hive.metastore.execute.setugi","hive.execution.engine");
+        HiveConf.ConfVars.METASTOREURIS.varname,
+        HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL.varname,
+        HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI.varname,
+        HiveConf.ConfVars.HIVE_EXECUTION_ENGINE.varname,
+        HiveConf.ConfVars.HIVE_CONF_HIDDEN_LIST.varname);
 
     //each items is a "key=value" format
     List<String> webhcatHiveProps = new ArrayList<String>(hiveProps());
@@ -232,6 +237,12 @@ public class AppConfig extends Configuration {
       hiveProps.append(hiveProps.length() > 0 ? "," : "").append(StringUtils.escapeString(whProp));
     }
     set(HIVE_PROPS_NAME, hiveProps.toString());
+    // Setting the hidden list
+    String hiddenProperties = hiveConf.get(HiveConf.ConfVars.HIVE_CONF_HIDDEN_LIST.varname);
+    if (this.get(HiveConf.ConfVars.HIVE_CONF_HIDDEN_LIST.varname) == null
+        && hiddenProperties!=null) {
+      set(HiveConf.ConfVars.HIVE_CONF_HIDDEN_LIST.varname, hiddenProperties);
+    }
   }
 
   private static void logConfigLoadAttempt(String path) {

http://git-wip-us.apache.org/repos/asf/hive/blob/fa1d8c76/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/TempletonUtils.java
----------------------------------------------------------------------
diff --git a/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/TempletonUtils.java b/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/TempletonUtils.java
index 201e647..0b6301c 100644
--- a/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/TempletonUtils.java
+++ b/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/TempletonUtils.java
@@ -42,6 +42,7 @@ import java.util.regex.Pattern;
 
 import javax.ws.rs.core.UriBuilder;
 
+import org.apache.hadoop.hive.common.LogUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -484,7 +485,8 @@ public class TempletonUtils {
         }
       }
       else {
-        sb.append(propKey).append('=').append(map.get(propKey)).append('\n');
+        sb.append(propKey).append('=').append(LogUtils.maskIfPassword(propKey, map.get(propKey)));
+        sb.append('\n');
       }
     }
     return sb.append("END").append(header).append('\n');

http://git-wip-us.apache.org/repos/asf/hive/blob/fa1d8c76/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 b4c6982..fd64056 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
@@ -227,13 +227,6 @@ public final class Utilities {
   public static String REDUCENAME = "Reducer ";
 
   /**
-   * Constants for log masking
-   */
-  private static String KEY_TO_MASK_WITH = "password";
-  private static String MASKED_VALUE = "###_MASKED_###";
-
-
-  /**
    * ReduceField:
    * KEY: record key
    * VALUE: record value
@@ -3632,22 +3625,6 @@ public final class Utilities {
   }
 
   /**
-   * Returns MASKED_VALUE if the key contains KEY_TO_MASK_WITH or the original property otherwise.
-   * Used to mask environment variables, and properties in logs which contain passwords
-   * @param key The property key to check
-   * @param value The original value of the property
-   * @return The masked property value
-   */
-  public static String maskIfPassword(String key, String value) {
-    if (key!=null && value!=null) {
-      if (key.toLowerCase().indexOf(KEY_TO_MASK_WITH) != -1) {
-        return MASKED_VALUE;
-      }
-    }
-    return value;
-  }
-
-  /**
    * Check if LLAP IO supports the column type that is being read
    * @param conf - configuration
    * @return false for types not supported by vectorization, true otherwise

http://git-wip-us.apache.org/repos/asf/hive/blob/fa1d8c76/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
index ac922ce..48d2540 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
@@ -41,6 +41,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.LogUtils;
 import org.apache.hadoop.hive.common.io.CachingPrintStream;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -309,7 +310,7 @@ public class MapredLocalTask extends Task<MapredLocalWork> implements Serializab
         String name = entry.getKey();
         String value = entry.getValue();
         env[pos++] = name + "=" + value;
-        LOG.debug("Setting env: " + name + "=" + Utilities.maskIfPassword(name, value));
+        LOG.debug("Setting env: " + name + "=" + LogUtils.maskIfPassword(name, value));
       }
 
       LOG.info("Executing: " + cmdLine);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa1d8c76/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java
index 8e5c0da..784b9c9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java
@@ -28,6 +28,7 @@ import java.util.Set;
 
 import org.apache.commons.compress.utils.CharsetNames;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hive.common.LogUtils;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.slf4j.Logger;
@@ -99,7 +100,7 @@ public class HiveSparkClientFactory {
             sparkConf.put(propertyName, properties.getProperty(propertyName));
             LOG.info(String.format(
               "load spark property from %s (%s -> %s).",
-              SPARK_DEFAULT_CONF_FILE, propertyName, Utilities.maskIfPassword(propertyName,value)));
+              SPARK_DEFAULT_CONF_FILE, propertyName, LogUtils.maskIfPassword(propertyName,value)));
           }
         }
       }
@@ -136,7 +137,7 @@ public class HiveSparkClientFactory {
         sparkConf.put(propertyName, value);
         LOG.info(String.format(
           "load spark property from hive configuration (%s -> %s).",
-          propertyName, Utilities.maskIfPassword(propertyName,value)));
+          propertyName, LogUtils.maskIfPassword(propertyName,value)));
       } else if (propertyName.startsWith("yarn") &&
         (sparkMaster.equals("yarn-client") || sparkMaster.equals("yarn-cluster"))) {
         String value = hiveConf.get(propertyName);
@@ -146,7 +147,7 @@ public class HiveSparkClientFactory {
         sparkConf.put("spark.hadoop." + propertyName, value);
         LOG.info(String.format(
           "load yarn property from hive configuration in %s mode (%s -> %s).",
-          sparkMaster, propertyName, Utilities.maskIfPassword(propertyName,value)));
+          sparkMaster, propertyName, LogUtils.maskIfPassword(propertyName,value)));
       } else if (propertyName.equals(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY)) {
         String value = hiveConf.get(propertyName);
         if (value != null && !value.isEmpty()) {
@@ -159,7 +160,7 @@ public class HiveSparkClientFactory {
         String value = hiveConf.get(propertyName);
         sparkConf.put("spark.hadoop." + propertyName, value);
         LOG.info(String.format(
-          "load HBase configuration (%s -> %s).", propertyName, Utilities.maskIfPassword(propertyName,value)));
+          "load HBase configuration (%s -> %s).", propertyName, LogUtils.maskIfPassword(propertyName,value)));
       }
 
       if (RpcConfiguration.HIVE_SPARK_RSC_CONFIGS.contains(propertyName)) {
@@ -167,7 +168,7 @@ public class HiveSparkClientFactory {
         sparkConf.put(propertyName, value);
         LOG.info(String.format(
           "load RPC property from hive configuration (%s -> %s).",
-          propertyName, Utilities.maskIfPassword(propertyName,value)));
+          propertyName, LogUtils.maskIfPassword(propertyName,value)));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/fa1d8c76/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
index 0786f9b..bd067aa 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
@@ -155,18 +155,6 @@ public class TestUtilities {
   }
 
   @Test
-  public void testMaskIfPassword() {
-    Assert.assertNull(Utilities.maskIfPassword("",null));
-    Assert.assertNull(Utilities.maskIfPassword(null,null));
-    Assert.assertEquals("test",Utilities.maskIfPassword(null,"test"));
-    Assert.assertEquals("test2",Utilities.maskIfPassword("any","test2"));
-    Assert.assertEquals("###_MASKED_###",Utilities.maskIfPassword("password","test3"));
-    Assert.assertEquals("###_MASKED_###",Utilities.maskIfPassword("a_passWord","test4"));
-    Assert.assertEquals("###_MASKED_###",Utilities.maskIfPassword("password_a","test5"));
-    Assert.assertEquals("###_MASKED_###",Utilities.maskIfPassword("a_PassWord_a","test6"));
-  }
-
-  @Test
   public void testRemoveTempOrDuplicateFilesOnTezNoDp() throws Exception {
     List<Path> paths = runRemoveTempOrDuplicateFilesTestCase("tez", false);
     assertEquals(0, paths.size());