PIG-4896: Param substitution ignored when redefined (knoguchi)
git-svn-id: https://svn.apache.org/repos/asf/pig/trunk@1748622 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/CHANGES.txt b/CHANGES.txt
index 5bfefae..9e2f4af 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -30,6 +30,8 @@
BUG FIXES
+PIG-4896: Param substitution ignored when redefined (knoguchi)
+
PIG-2315: Make as clause work in generate (daijy via knoguchi)
PIG-4921: Kill running jobs on InterruptedException (rohini)
diff --git a/src/org/apache/pig/tools/parameters/PreprocessorContext.java b/src/org/apache/pig/tools/parameters/PreprocessorContext.java
index 40f78b2..e5c2ef6 100644
--- a/src/org/apache/pig/tools/parameters/PreprocessorContext.java
+++ b/src/org/apache/pig/tools/parameters/PreprocessorContext.java
@@ -46,10 +46,6 @@
private Map<String, String> param_val;
- // used internally to detect when a param is set multiple times,
- // but it set with the same value so it's ok not to log a warning
- private Map<String, String> param_source;
-
private PigContext pigContext;
public Map<String, String> getParamVal() {
@@ -64,23 +60,16 @@
*/
public PreprocessorContext(int limit) {
param_val = new Hashtable<String, String> (limit);
- param_source = new Hashtable<String, String> (limit);
}
public PreprocessorContext(Map<String, String> paramVal) {
param_val = paramVal;
- param_source = new Hashtable<String, String>(paramVal);
}
public void setPigContext(PigContext context) {
this.pigContext = context;
}
- /*
- public void processLiteral(String key, String val) {
- processLiteral(key, val, true);
- } */
-
/**
* This method generates parameter value by running specified command
*
@@ -102,21 +91,6 @@
processOrdLine(key, val, true);
}
- /*
- public void processLiteral(String key, String val, Boolean overwrite) {
-
- if (param_val.containsKey(key)) {
- if (overwrite) {
- log.warn("Warning : Multiple values found for " + key + ". Using value " + val);
- } else {
- return;
- }
- }
-
- String sub_val = substitute(val);
- param_val.put(key, sub_val);
- } */
-
/**
* This method generates parameter value by running specified command
*
@@ -129,20 +103,20 @@
filter.validate(PigCommandFilter.Command.SH);
}
- if (param_val.containsKey(key)) {
- if (param_source.get(key).equals(val) || !overwrite) {
- return;
- } else {
- log.warn("Warning : Multiple values found for " + key
- + ". Using value " + val);
- }
+ if (param_val.containsKey(key) && !overwrite) {
+ return;
}
- param_source.put(key, val);
-
val = val.substring(1, val.length()-1); //to remove the backticks
String sub_val = substitute(val);
sub_val = executeShellCommand(sub_val);
+
+ if (param_val.containsKey(key) && !param_val.get(key).equals(sub_val) ) {
+ //(boolean overwrite is always true here)
+ log.warn("Warning : Multiple values found for " + key + " command `" + val + "`. "
+ + "Previous value " + param_val.get(key) + ", now using value " + sub_val);
+ }
+
param_val.put(key, sub_val);
}
@@ -175,17 +149,17 @@
*/
public void processOrdLine(String key, String val, Boolean overwrite) throws ParameterSubstitutionException {
+ String sub_val = substitute(val, key);
if (param_val.containsKey(key)) {
- if (param_source.get(key).equals(val) || !overwrite) {
+ if (param_val.get(key).equals(sub_val) || !overwrite) {
return;
} else {
- log.warn("Warning : Multiple values found for " + key + ". Using value " + val);
+ log.warn("Warning : Multiple values found for " + key
+ + ". Previous value " + param_val.get(key)
+ + ", now using value " + sub_val);
}
}
- param_source.put(key, val);
-
- String sub_val = substitute(val, key);
param_val.put(key, sub_val);
}
diff --git a/test/org/apache/pig/test/TestParamSubPreproc.java b/test/org/apache/pig/test/TestParamSubPreproc.java
index f9af2c6..4b9d18c 100644
--- a/test/org/apache/pig/test/TestParamSubPreproc.java
+++ b/test/org/apache/pig/test/TestParamSubPreproc.java
@@ -19,6 +19,7 @@
package org.apache.pig.test;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@@ -943,6 +944,79 @@
log.info("Done");
}
+ @Test
+ public void testSubstitutionWithRedeclaration() throws Exception{
+ log.info("Starting test testSubstitutionWithRedeclaration()");
+ final String queryString =
+ "%declare output '/tmp/abc';\n" +
+ "%declare actualoutput '$output.out';\n" +
+ "A = load 'input.txt' ;\n" +
+ "store A into '$actualoutput';\n" +
+ "%declare output '/tmp/def';\n" +
+ "%declare actualoutput '$output.out';\n" +
+ "store A into '$actualoutput';";
+
+
+ ParameterSubstitutionPreprocessor ps = new ParameterSubstitutionPreprocessor(50);
+ pigIStream = new BufferedReader(
+ new InputStreamReader(new ByteArrayInputStream(queryString.getBytes("UTF-8"))));
+ pigOStream = new FileWriter(basedir + "/output1.pig");
+
+ String[] arg = {"output = 'output.txt'"};
+ String[] argFiles = null;
+ ps.genSubstitutedFile(pigIStream , pigOStream , arg , argFiles);
+
+ FileInputStream pigResultStream = new FileInputStream(basedir + "/output1.pig");
+
+ String expectedString = queryString.replaceAll("%declare [0-9a-zA-Z.'/\\$; ]*\n",";\n")
+ .replaceAll("\\$","")
+ .replaceFirst("actualoutput","/tmp/abc.out")
+ .replaceFirst("actualoutput","/tmp/def.out");
+ InputStream expected = new ByteArrayInputStream(expectedString.getBytes("UTF-8"));
+
+ compareResults(expected, pigResultStream);
+
+ log.info("Done");
+ }
+
+ @Test
+ public void testSubstitutionWithRedeclaredShell() throws Exception{
+ log.info("Starting test testSubstitutionWithRedeclaredShell()");
+ final String queryString =
+ "A = load 'input.txt' ;\n" +
+ "%declare now `bash -c \"date +'%Y%m%d_%H:%M:%S'; sleep 1;\"`;\n" +
+ "store A into '$now';\n" +
+ "%declare now `bash -c \"date +'%Y%m%d_%H:%M:%S'; sleep 1;\"`;\n" +
+ "store A into '$now';\n";
+
+ ParameterSubstitutionPreprocessor ps = new ParameterSubstitutionPreprocessor(50);
+ pigIStream = new BufferedReader(
+ new InputStreamReader(new ByteArrayInputStream(queryString.getBytes("UTF-8"))));
+ pigOStream = new FileWriter(basedir + "/output1.pig");
+
+ String[] arg = {"output = 'output.txt'"};
+ String[] argFiles = null;
+ ps.genSubstitutedFile(pigIStream , pigOStream , arg , argFiles);
+
+ BufferedReader pigresult = new BufferedReader(new InputStreamReader(new FileInputStream(basedir + "/output1.pig")));
+
+
+ String [] filenames = new String [2];
+ int index=0;
+ String line;
+ while ((line = pigresult.readLine())!=null) {
+ if( line.startsWith("store A into") ) {
+ filenames[index++] = line.split(" ")[3];
+ }
+ }
+
+ assertEquals("There should be 2 store statements", 2, index);
+ assertNotEquals("Identical shell param should be reexecuted.",
+ filenames[0],
+ filenames[1]);
+ log.info("Done");
+ }
+
@SuppressWarnings("resource")
private BufferedReader WithConditionalReplacement(String filename, String orig, String dest, boolean replace) throws IOException {
BufferedReader pigOrigIStream = new BufferedReader(new FileReader(filename));