You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2015/08/25 23:29:04 UTC

[34/50] [abbrv] hive git commit: HIVE-8583 Cleanup & Test for hive.script.operator.env.blacklist (Lars Franke via gates)

HIVE-8583 Cleanup & Test for hive.script.operator.env.blacklist (Lars Franke via gates)


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

Branch: refs/heads/llap
Commit: d78bb7c49d88a8acf3fd317a06f9563003571f4c
Parents: 87930e2
Author: Alan Gates <ga...@hortonworks.com>
Authored: Fri Aug 21 10:09:22 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Fri Aug 21 10:09:22 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/ScriptOperator.java     | 85 ++++++++++----------
 .../hadoop/hive/ql/exec/TestOperators.java      | 16 ++++
 2 files changed, 60 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d78bb7c4/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java
index f2eed44..82ec668 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java
@@ -18,25 +18,6 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.Timer;
-import java.util.TimerTask;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -57,6 +38,26 @@ import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.spark.SparkFiles;
 
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
 /**
  * ScriptOperator.
  *
@@ -146,16 +147,19 @@ public class ScriptOperator extends Operator<ScriptDesc> implements
     return value;
   }
 
-  boolean blackListed(String name) {
+  /**
+   * Checks whether a given configuration name is blacklisted and should not be converted
+   * to an environment variable.
+   */
+  boolean blackListed(Configuration conf, String name) {
     if (blackListedConfEntries == null) {
       blackListedConfEntries = new HashSet<String>();
-      if (hconf != null) {
-        String bl = hconf.get(HiveConf.ConfVars.HIVESCRIPT_ENV_BLACKLIST.toString());
-        if (bl != null && bl.length() > 0) {
+      if (conf != null) {
+        String bl = conf.get(HiveConf.ConfVars.HIVESCRIPT_ENV_BLACKLIST.toString(),
+          HiveConf.ConfVars.HIVESCRIPT_ENV_BLACKLIST.getDefaultValue());
+        if (bl != null && !bl.isEmpty()) {
           String[] bls = bl.split(",");
-          for (String b : bls) {
-            blackListedConfEntries.add(b);
-          }
+          Collections.addAll(blackListedConfEntries, bls);
         }
       }
     }
@@ -171,7 +175,7 @@ public class ScriptOperator extends Operator<ScriptDesc> implements
     while (it.hasNext()) {
       Map.Entry<String, String> en = it.next();
       String name = en.getKey();
-      if (!blackListed(name)) {
+      if (!blackListed(conf, name)) {
         // String value = (String)en.getValue(); // does not apply variable
         // expansion
         String value = conf.get(name); // does variable expansion
@@ -306,8 +310,7 @@ public class ScriptOperator extends Operator<ScriptDesc> implements
   void displayBrokenPipeInfo() {
     if (isLogInfoEnabled) {
       LOG.info("The script did not consume all input data. This is considered as an error.");
-      LOG.info("set " + HiveConf.ConfVars.ALLOWPARTIALCONSUMP.toString()
-	  + "=true; to ignore it.");
+      LOG.info("set " + HiveConf.ConfVars.ALLOWPARTIALCONSUMP.toString() + "=true; to ignore it.");
     }
     return;
   }
@@ -349,12 +352,12 @@ public class ScriptOperator extends Operator<ScriptDesc> implements
         }
 
         String[] wrappedCmdArgs = addWrapper(cmdArgs);
-	if (isLogInfoEnabled) {
-	  LOG.info("Executing " + Arrays.asList(wrappedCmdArgs));
-	  LOG.info("tablename=" + tableName);
-	  LOG.info("partname=" + partitionName);
-	  LOG.info("alias=" + alias);
-	}
+        if (isLogInfoEnabled) {
+          LOG.info("Executing " + Arrays.asList(wrappedCmdArgs));
+          LOG.info("tablename=" + tableName);
+          LOG.info("partname=" + partitionName);
+          LOG.info("alias=" + alias);
+        }
 
         ProcessBuilder pb = new ProcessBuilder(wrappedCmdArgs);
         Map<String, String> env = pb.environment();
@@ -672,9 +675,9 @@ public class ScriptOperator extends Operator<ScriptDesc> implements
       long now = System.currentTimeMillis();
       // reporter is a member variable of the Operator class.
       if (now - lastReportTime > 60 * 1000 && reporter != null) {
-	if (isLogInfoEnabled) {
-	  LOG.info("ErrorStreamProcessor calling reporter.progress()");
-	}
+        if (isLogInfoEnabled) {
+          LOG.info("ErrorStreamProcessor calling reporter.progress()");
+        }
         lastReportTime = now;
         reporter.progress();
       }
@@ -730,9 +733,9 @@ public class ScriptOperator extends Operator<ScriptDesc> implements
           }
           proc.processLine(row);
         }
-	if (isLogInfoEnabled) {
-	  LOG.info("StreamThread " + name + " done");
-	}
+        if (isLogInfoEnabled) {
+          LOG.info("StreamThread " + name + " done");
+        }
 
       } catch (Throwable th) {
         scriptError = th;

http://git-wip-us.apache.org/repos/asf/hive/blob/d78bb7c4/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java
index c3a36c0..21dcf86 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.TextInputFormat;
+import org.junit.Assert;
 import org.junit.Test;
 
 /**
@@ -192,6 +193,21 @@ public class TestOperators extends TestCase {
     }
   }
 
+  public void testScriptOperatorBlacklistedEnvVarsProcessing() {
+    ScriptOperator scriptOperator = new ScriptOperator();
+
+    Configuration hconf = new JobConf(ScriptOperator.class);
+
+    Map<String, String> env = new HashMap<String, String>();
+
+    HiveConf.setVar(hconf, HiveConf.ConfVars.HIVESCRIPT_ENV_BLACKLIST, "foobar");
+    hconf.set("foobar", "foobar");
+    hconf.set("barfoo", "barfoo");
+    scriptOperator.addJobConfToEnvironment(hconf, env);
+    Assert.assertFalse(env.containsKey("foobar"));
+    Assert.assertTrue(env.containsKey("barfoo"));
+  }
+
   public void testScriptOperator() throws Throwable {
     try {
       System.out.println("Testing Script Operator");