You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by an...@apache.org on 2014/03/06 23:50:02 UTC
svn commit: r1575083 - in /pig/trunk: CHANGES.txt
src/org/apache/pig/PigConstants.java
src/org/apache/pig/backend/hadoop/datastorage/ConfigurationUtil.java
src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
Author: aniket486
Date: Thu Mar 6 22:50:02 2014
New Revision: 1575083
URL: http://svn.apache.org/r1575083
Log:
PIG-3731: Ability to specify local-mode specific configuration (useful for local/auto-local mode) (aniket486)
Modified:
pig/trunk/CHANGES.txt
pig/trunk/src/org/apache/pig/PigConstants.java
pig/trunk/src/org/apache/pig/backend/hadoop/datastorage/ConfigurationUtil.java
pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1575083&r1=1575082&r2=1575083&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Thu Mar 6 22:50:02 2014
@@ -30,6 +30,8 @@ PIG-2207: Support custom counters for ag
IMPROVEMENTS
+PIG-3731: Ability to specify local-mode specific configuration (useful for local/auto-local mode) (aniket486)
+
PIG-3793: Provide info on number of LogicalRelationalOperator(s) used in the script through LogicalPlanData (prkommireddi)
PIG-3778: Log list of running jobs along with progress (rohini)
Modified: pig/trunk/src/org/apache/pig/PigConstants.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/PigConstants.java?rev=1575083&r1=1575082&r2=1575083&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/PigConstants.java (original)
+++ pig/trunk/src/org/apache/pig/PigConstants.java Thu Mar 6 22:50:02 2014
@@ -47,4 +47,9 @@ public class PigConstants {
*/
public static final String PIG_OPTIMIZER_RULES_DISABLED_KEY = "pig.optimizer.rules.disabled";
+ /**
+ * Prefix used by pig to configure local mode configuration
+ */
+ public static final String PIG_LOCAL_CONF_PREFIX = "pig.local.";
+
}
\ No newline at end of file
Modified: pig/trunk/src/org/apache/pig/backend/hadoop/datastorage/ConfigurationUtil.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/datastorage/ConfigurationUtil.java?rev=1575083&r1=1575082&r2=1575083&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/datastorage/ConfigurationUtil.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/datastorage/ConfigurationUtil.java Thu Mar 6 22:50:02 2014
@@ -26,6 +26,7 @@ import java.util.Properties;
import org.apache.hadoop.conf.Configuration;
import org.apache.pig.ExecType;
+import org.apache.pig.PigConstants;
import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce;
import org.apache.pig.backend.hadoop.executionengine.shims.HadoopShims;
import org.apache.pig.backend.hadoop.executionengine.util.MapRedUtil;
@@ -85,4 +86,15 @@ public class ConfigurationUtil {
Properties props = ConfigurationUtil.toProperties(localConf);
return props;
}
+
+ public static void replaceConfigForLocalMode(Configuration configuration) {
+ for (Entry<String, String> entry : configuration) {
+ String key = entry.getKey();
+ String value = entry.getValue();
+ if(key.startsWith(PigConstants.PIG_LOCAL_CONF_PREFIX)) {
+ String realConfKey = key.substring(PigConstants.PIG_LOCAL_CONF_PREFIX.length());
+ configuration.set(realConfKey, value);
+ }
+ }
+ }
}
Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java?rev=1575083&r1=1575082&r2=1575083&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java Thu Mar 6 22:50:02 2014
@@ -584,6 +584,10 @@ public class JobControlCompiler{
// we don't want to change fs settings back
continue;
}
+ if (key.startsWith("io.")) {
+ // we don't want to change io settings back
+ continue;
+ }
String value = entry.getValue();
if (conf.get(key) == null || !conf.get(key).equals(value)) {
conf.set(key, value);
@@ -616,6 +620,10 @@ public class JobControlCompiler{
conf.set("mapred.jar", submitJarFile.getPath());
}
}
+
+ if(isLocal(pigContext, conf)) {
+ ConfigurationUtil.replaceConfigForLocalMode(conf);
+ }
conf.set("pig.inputs", ObjectSerializer.serialize(inp));
conf.set("pig.inpTargets", ObjectSerializer.serialize(inpTargets));
conf.set("pig.inpSignatures", ObjectSerializer.serialize(inpSignatureLists));