You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ro...@apache.org on 2014/11/10 00:05:01 UTC

svn commit: r1637758 [1/2] - in /pig/trunk: ./ shims/test/hadoop23/org/apache/pig/test/ src/org/apache/pig/ src/org/apache/pig/backend/hadoop/executionengine/fetch/ src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/ src/org/apache/pig/ba...

Author: rohini
Date: Sun Nov  9 23:04:59 2014
New Revision: 1637758

URL: http://svn.apache.org/r1637758
Log:
PIG-4318: Make PigConfiguration naming consistent (rohini)

Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/shims/test/hadoop23/org/apache/pig/test/TezMiniCluster.java
    pig/trunk/src/org/apache/pig/Main.java
    pig/trunk/src/org/apache/pig/PigConfiguration.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/fetch/FetchOptimizer.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigRecordReader.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPartialAgg.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezJob.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/TezCompiler.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/optimizer/ParallelismSetter.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/AccumulatorOptimizerUtil.java
    pig/trunk/src/org/apache/pig/data/SchemaTupleBackend.java
    pig/trunk/src/org/apache/pig/data/SchemaTupleClassGenerator.java
    pig/trunk/src/org/apache/pig/data/SchemaTupleFrontend.java
    pig/trunk/src/org/apache/pig/impl/builtin/PoissonSampleLoader.java
    pig/trunk/src/org/apache/pig/impl/util/PropertiesUtil.java
    pig/trunk/src/org/apache/pig/tools/pigstats/ScriptState.java
    pig/trunk/test/org/apache/pig/data/TestSchemaTuple.java
    pig/trunk/test/org/apache/pig/test/TestAccumulator.java
    pig/trunk/test/org/apache/pig/test/TestAutoLocalMode.java
    pig/trunk/test/org/apache/pig/test/TestBatchAliases.java
    pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java
    pig/trunk/test/org/apache/pig/test/TestFetch.java
    pig/trunk/test/org/apache/pig/test/TestGrunt.java
    pig/trunk/test/org/apache/pig/test/TestLoad.java
    pig/trunk/test/org/apache/pig/test/TestMultiQuery.java
    pig/trunk/test/org/apache/pig/test/TestMultiQueryBasic.java
    pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java
    pig/trunk/test/org/apache/pig/test/TestMultiQueryLocal.java
    pig/trunk/test/org/apache/pig/test/TestPOPartialAggPlan.java
    pig/trunk/test/org/apache/pig/test/TestPigRunner.java
    pig/trunk/test/org/apache/pig/test/TestPigServer.java
    pig/trunk/test/org/apache/pig/test/TestPredeployedJar.java
    pig/trunk/test/org/apache/pig/test/TestStore.java
    pig/trunk/test/org/apache/pig/tez/TestSecondarySortTez.java
    pig/trunk/test/org/apache/pig/tez/TestTezCompiler.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Sun Nov  9 23:04:59 2014
@@ -37,6 +37,8 @@ INCOMPATIBLE CHANGES
  
 IMPROVEMENTS
 
+PIG-4318: Make PigConfiguration naming consistent (rohini)
+
 PIG-4316: Port TestHBaseStorage to tez local mode (rohini)
 
 PIG-4224: Upload Tez payload history string to timeline server (daijy)

Modified: pig/trunk/shims/test/hadoop23/org/apache/pig/test/TezMiniCluster.java
URL: http://svn.apache.org/viewvc/pig/trunk/shims/test/hadoop23/org/apache/pig/test/TezMiniCluster.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/shims/test/hadoop23/org/apache/pig/test/TezMiniCluster.java (original)
+++ pig/trunk/shims/test/hadoop23/org/apache/pig/test/TezMiniCluster.java Sun Nov  9 23:04:59 2014
@@ -144,7 +144,7 @@ public class TezMiniCluster extends Mini
 
             m_conf = m_mr_conf;
             // Turn FetchOptimizer off so that we can actually test Tez
-            m_conf.set(PigConfiguration.OPT_FETCH, System.getProperty("test.opt.fetch", "false"));
+            m_conf.set(PigConfiguration.PIG_OPT_FETCH, System.getProperty("test.opt.fetch", "false"));
 
             System.setProperty("junit.hadoop.conf", CONF_DIR.getPath());
             System.setProperty("hadoop.log.dir", "build/test/logs");

Modified: pig/trunk/src/org/apache/pig/Main.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/Main.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/Main.java (original)
+++ pig/trunk/src/org/apache/pig/Main.java Sun Nov  9 23:04:59 2014
@@ -316,11 +316,11 @@ public class Main {
 
                 case 'M':
                     // turns off multiquery optimization
-                    properties.setProperty(PigConfiguration.OPT_MULTIQUERY,""+false);
+                    properties.setProperty(PigConfiguration.PIG_OPT_MULTIQUERY,""+false);
                     break;
 
                 case 'N':
-                    properties.setProperty(PigConfiguration.OPT_FETCH,""+false);
+                    properties.setProperty(PigConfiguration.PIG_OPT_FETCH,""+false);
                     break;
 
                 case 'p':

Modified: pig/trunk/src/org/apache/pig/PigConfiguration.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/PigConfiguration.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/PigConfiguration.java (original)
+++ pig/trunk/src/org/apache/pig/PigConfiguration.java Sun Nov  9 23:04:59 2014
@@ -29,94 +29,123 @@ public class PigConfiguration {
     /////////////////////////       COMMAND LINE KEYS       /////////////////////////////
     /////////////////////////////////////////////////////////////////////////////////////
 
+    // Pig runtime optimizations
     /**
-     * Controls the fraction of total memory that is allowed to be used by
-     * cached bags. Default is 0.2.
+     * This key is to turn on auto local mode feature
      */
-    public static final String PIG_CACHEDBAG_MEMUSAGE = "pig.cachedbag.memusage";
-    
+    public static final String PIG_AUTO_LOCAL_ENABLED = "pig.auto.local.enabled";
     /**
-     * Configurations for specifying alternate implementations for cached bags
+     * Controls the max threshold size to convert jobs to run in local mode
      */
-    public static final String PIG_CACHEDBAG_TYPE = "pig.cachedbag.type";
-    public static final String PIG_CACHEDBAG_DISTINCT_TYPE = "pig.cachedbag.distinct.type";
-    public static final String PIG_CACHEDBAG_SORT_TYPE = "pig.cachedbag.sort.type";
+    public static final String PIG_AUTO_LOCAL_INPUT_MAXBYTES = "pig.auto.local.input.maxbytes";
+
+    /**
+     * Boolean value used to enable or disable fetching without a mapreduce job for DUMP. True by default
+     */
+    public static final String PIG_OPT_FETCH = "opt.fetch";
 
+    // Pig query planning and execution optimizations
+    /**
+     * Boolean value used to enable or disable multiquery optimization. True by default
+     */
+    public static final String PIG_OPT_MULTIQUERY = "opt.multiquery";
 
-    public static final String ACCUMULATIVE_BATCHSIZE = "pig.accumulative.batchsize";
+    /**
+     * Boolean value used to enable or disable accumulator optimization. True by default
+     */
+    public static final String PIG_OPT_ACCUMULATOR = "opt.accumulator";
+    public static final String PIG_ACCUMULATIVE_BATCHSIZE = "pig.accumulative.batchsize";
 
     /**
-     * Controls whether partial aggregation is turned on
+     * This key is used to enable or disable union optimization in tez. True by default
      */
-    public static final String PROP_EXEC_MAP_PARTAGG = "pig.exec.mapPartAgg";
+    public static final String PIG_TEZ_OPT_UNION = "pig.tez.opt.union";
 
     /**
+     * Boolean value to enable or disable partial aggregation in map. Disabled by default
+     */
+    public static final String PIG_EXEC_MAP_PARTAGG = "pig.exec.mapPartAgg";
+    /**
      * Controls the minimum reduction in-mapper Partial Aggregation should achieve in order
      * to stay on. If after a period of observation this reduction is not achieved,
      * in-mapper aggregation will be turned off and a message logged to that effect.
      */
-    public static final String PARTAGG_MINREDUCTION = "pig.exec.mapPartAgg.minReduction";
+    public static final String PIG_EXEC_MAP_PARTAGG_MINREDUCTION = "pig.exec.mapPartAgg.minReduction";
 
     /**
-     * Controls whether execution time of Pig UDFs should be tracked.
-     * This feature uses counters; use judiciously.
+     * Boolean value to enable or disable use of combiners in MapReduce jobs. Enabled by default
      */
-    public static final String TIME_UDFS = "pig.udf.profile";
-    public static final String TIME_UDFS_FREQUENCY = "pig.udf.profile.frequency";
+    public static final String PIG_EXEC_NO_COMBINER = "pig.exec.nocombiner";
 
     /**
-     * This key must be set to true by the user for code generation to be used.
-     * In the future, it may be turned on by default (at least in certain cases),
-     * but for now it is too experimental.
+     * This key controls whether secondary sort key is used for optimization in case
+     * of nested distinct or sort
      */
-    public static final String SHOULD_USE_SCHEMA_TUPLE = "pig.schematuple";
-
-    public static final String SCHEMA_TUPLE_SHOULD_USE_IN_UDF = "pig.schematuple.udf";
-
-    public static final String SCHEMA_TUPLE_SHOULD_USE_IN_FOREACH = "pig.schematuple.foreach";
-
-    public static final String SCHEMA_TUPLE_SHOULD_USE_IN_FRJOIN = "pig.schematuple.fr_join";
-
-    public static final String SCHEMA_TUPLE_SHOULD_USE_IN_MERGEJOIN = "pig.schematuple.merge_join";
-
-    public static final String SCHEMA_TUPLE_SHOULD_ALLOW_FORCE = "pig.schematuple.force";
+    public static final String PIG_EXEC_NO_SECONDARY_KEY = "pig.exec.nosecondarykey";
 
+    // Pig memory usage control settings
     /**
-     * This key is used to enable multiquery optimization.
+     * Controls the fraction of total memory that is allowed to be used by
+     * cached bags. Default is 0.2.
      */
-    public static final String OPT_MULTIQUERY = "opt.multiquery";
+    public static final String PIG_CACHEDBAG_MEMUSAGE = "pig.cachedbag.memusage";
 
     /**
-     * This key is used to enable accumulator optimization.
+     * % of memory available for the input data. This is currently equal to the
+     * memory available for the skewed join
      */
-    public static final String OPT_ACCUMULATOR = "opt.accumulator";
-
+    public static final String PIG_SKEWEDJOIN_REDUCE_MEMUSAGE = "pig.skewedjoin.reduce.memusage";
 
     /**
-     * This key is used to configure auto parallelism in tez. Default is true.
+     * This key used to control the maximum size loaded into
+     * the distributed cache when doing fragment-replicated join
      */
-    public static final String TEZ_AUTO_PARALLELISM = "pig.tez.auto.parallelism";
+    public static final String PIG_JOIN_REPLICATED_MAX_BYTES = "pig.join.replicated.max.bytes";
 
+    // Pig cached bag type settings
     /**
-     * This key is used to enable union optimization.
+     * Configurations for specifying alternate implementations for cached bags. Rarely used
      */
-    public static final String TEZ_OPT_UNION = "pig.tez.opt.union";
+    public static final String PIG_CACHEDBAG_TYPE = "pig.cachedbag.type";
+    public static final String PIG_CACHEDBAG_DISTINCT_TYPE = "pig.cachedbag.distinct.type";
+    public static final String PIG_CACHEDBAG_SORT_TYPE = "pig.cachedbag.sort.type";
 
+    // Pig reducer parallelism estimation settings
+    public static final String PIG_EXEC_REDUCER_ESTIMATOR = "pig.exec.reducer.estimator";
+    public static final String PIG_EXEC_REDUCER_ESTIMATOR_CONSTRUCTOR_ARG_KEY =  "pig.exec.reducer.estimator.arg";
     /**
-     * This key is used to define whether to reuse AM in Tez jobs.
+     * This key is used to configure auto parallelism in tez. Default is true.
      */
-    public static final String TEZ_SESSION_REUSE = "pig.tez.session.reuse";
+    public static final String PIG_TEZ_AUTO_PARALLELISM = "pig.tez.auto.parallelism";
 
+    // Pig UDF profiling settings
     /**
-     * This key is used to configure the interval of dag status report in seconds.
+     * Controls whether execution time of Pig UDFs should be tracked.
+     * This feature uses counters; use judiciously.
      */
-    public static final String TEZ_DAG_STATUS_REPORT_INTERVAL = "pig.tez.dag.status.report.interval";
+    public static final String PIG_UDF_PROFILE = "pig.udf.profile";
+    public static final String PIG_UDF_PROFILE_FREQUENCY = "pig.udf.profile.frequency";
 
+    // Pig schema tuple settings
     /**
-     * Turns off use of combiners in MapReduce jobs produced by Pig.
+     * This key must be set to true by the user for code generation to be used.
+     * In the future, it may be turned on by default (at least in certain cases),
+     * but for now it is too experimental.
      */
-    public static final String PROP_NO_COMBINER = "pig.exec.nocombiner";
+    public static final String PIG_SCHEMA_TUPLE_ENABLED = "pig.schematuple";
+
+    public static final String PIG_SCHEMA_TUPLE_USE_IN_UDF = "pig.schematuple.udf";
+
+    public static final String PIG_SCHEMA_TUPLE_USE_IN_FOREACH = "pig.schematuple.foreach";
+
+    public static final String PIG_SCHEMA_TUPLE_USE_IN_FRJOIN = "pig.schematuple.fr_join";
 
+    public static final String PIG_SCHEMA_TUPLE_USE_IN_MERGEJOIN = "pig.schematuple.merge_join";
+
+    public static final String PIG_SCHEMA_TUPLE_ALLOW_FORCE = "pig.schematuple.force";
+
+
+    // Pig Streaming settings
     /**
      * This key can be used to defined what properties will be set in the streaming environment.
      * Just set this property to a comma-delimited list of properties to set, and those properties
@@ -130,33 +159,41 @@ public class PigConfiguration {
      */
     public static final String PIG_STREAMING_UDF_PYTHON_COMMAND = "pig.streaming.udf.python.command";
 
+    // Pig input format settings
     /**
-     * This key is used to define the default load func. Pig will fallback on PigStorage
-     * as default in case this is undefined.
+     * Turns combine split files on or off
      */
-    public static final String PIG_DEFAULT_LOAD_FUNC = "pig.default.load.func";
+    public static final String PIG_SPLIT_COMBINATION = "pig.splitCombination";
 
     /**
-     * This key is used to define the default store func. Pig will fallback on PigStorage
-     * as default in case this is undefined.
+     * Whether turns combine split files off. This is for internal use only
      */
-    public static final String PIG_DEFAULT_STORE_FUNC = "pig.default.store.func";
+    public static final String PIG_NO_SPLIT_COMBINATION = "pig.noSplitCombination";
 
     /**
+     * Specifies the size, in bytes, of data to be processed by a single map.
+     * Smaller files are combined untill this size is reached.
+     */
+    public static final String PIG_MAX_COMBINED_SPLIT_SIZE = "pig.maxCombinedSplitSize";
+
+    // Pig output format settings
+    /**
+     * This key is used to define whether PigOutputFormat will be wrapped with LazyOutputFormat
+     * so that jobs won't write empty part files if no output is generated
+     */
+    public static final String PIG_OUTPUT_LAZY = "pig.output.lazy";
+    /**
      * This key is used to define whether to support recovery to handle the
      * application master getting restarted.
      */
     public static final String PIG_OUTPUT_COMMITTER_RECOVERY = "pig.output.committer.recovery.support";
 
-    /**
-     * This key is used to turn off the inclusion of settings in the jobs.
-     */
-    public static final String INSERT_ENABLED = "pig.script.info.enabled";
+    //Pig intermediate temporary file settings
 
     /**
-     * Controls the size of Pig script stored in job xml.
+     * Location where pig stores temporary files for job setup
      */
-    public static final String MAX_SCRIPT_SIZE = "pig.script.max.size";
+    public static final String PIG_TEMP_DIR = "pig.temp.dir";
 
     /**
      * This key is used to define whether to have intermediate file compressed
@@ -181,44 +218,12 @@ public class PigConfiguration {
      */
     public static final String PIG_DELETE_TEMP_FILE = "pig.delete.temp.files";
 
-    /**
-     * For a given mean and a confidence, a sample rate is obtained from a poisson udf
-     */
-    public static final String SAMPLE_RATE = "pig.sksampler.samplerate";
-
-    /**
-     * % of memory available for the input data. This is currently equal to the
-     * memory available for the skewed join
-     */
-    public static final String PERC_MEM_AVAIL = "pig.skewedjoin.reduce.memusage";
-
-    /**
-     * This key used to control the maximum size loaded into
-     * the distributed cache when doing fragment-replicated join
-     */
-    public static final String PIG_JOIN_REPLICATED_MAX_BYTES = "pig.join.replicated.max.bytes";
-
-    /**
-     * Turns combine split files on or off
-     */
-    public static final String PIG_SPLIT_COMBINATION = "pig.splitCombination";
-
-    /**
-     * Whether turns combine split files off. This is for internal use only
-     */
-    public static final String PIG_NO_SPLIT_COMBINATION = "pig.noSplitCombination";
-
-    /**
-     * Specifies the size, in bytes, of data to be processed by a single map.
-     * Smaller files are combined untill this size is reached.
-     */
-    public static final String PIG_MAX_COMBINED_SPLIT_SIZE = "pig.maxCombinedSplitSize";
 
+    //Pig skewedjoin and order by sampling settings
     /**
-     * This key controls whether secondary sort key is used for optimization in case
-     * of nested distinct or sort
+     * For a given mean and a confidence, a sample rate is obtained from a poisson udf
      */
-    public static final String PIG_EXEC_NO_SECONDARY_KEY = "pig.exec.nosecondarykey";
+    public static final String PIG_POISSON_SAMPLER_SAMPLE_RATE = "pig.sksampler.samplerate";
 
     /**
      * This key used to control the sample size of RandomeSampleLoader for
@@ -226,31 +231,29 @@ public class PigConfiguration {
      */
     public static final String PIG_RANDOM_SAMPLER_SAMPLE_SIZE = "pig.random.sampler.sample.size";
 
-    /**
-     * This key is to turn on auto local mode feature
-     */
-    public static final String PIG_AUTO_LOCAL_ENABLED = "pig.auto.local.enabled";
 
+    //Pig miscellaneous settings
     /**
-     * Controls the max threshold size to convert jobs to run in local mode
+     * This key is used to define the default load func. Pig will fallback on PigStorage
+     * as default in case this is undefined.
      */
-    public static final String PIG_AUTO_LOCAL_INPUT_MAXBYTES = "pig.auto.local.input.maxbytes";
+    public static final String PIG_DEFAULT_LOAD_FUNC = "pig.default.load.func";
 
     /**
-     * This parameter enables/disables fetching. By default it is turned on.
+     * This key is used to define the default store func. Pig will fallback on PigStorage
+     * as default in case this is undefined.
      */
-    public static final String OPT_FETCH = "opt.fetch";
+    public static final String PIG_DEFAULT_STORE_FUNC = "pig.default.store.func";
 
     /**
-     * This key is used to define whether PigOutputFormat will be wrapped with LazyOutputFormat
-     * so that jobs won't write empty part files if no output is generated
+     * This key is used to turn off the inclusion of settings in the jobs.
      */
-    public static final String PIG_OUTPUT_LAZY = "pig.output.lazy";
+    public static final String PIG_SCRIPT_INFO_ENABLED = "pig.script.info.enabled";
 
     /**
-     * Location where pig stores temporary files for job setup
+     * Controls the size of Pig script stored in job xml.
      */
-    public static final String PIG_TEMP_DIR = "pig.temp.dir";
+    public static final String PIG_SCRIPT_MAX_SIZE = "pig.script.max.size";
 
     /**
      * This key is turn on the user level cache
@@ -287,6 +290,80 @@ public class PigConfiguration {
      */
     public static final String PIG_NO_TASK_REPORT = "pig.stats.notaskreport";
 
-    public static final String REDUCER_ESTIMATOR_KEY = "pig.exec.reducer.estimator";
-    public static final String REDUCER_ESTIMATOR_ARG_KEY =  "pig.exec.reducer.estimator.arg";
+
+    // Pig on Tez runtime settings
+    /**
+     * This key is used to define whether to reuse AM in Tez jobs.
+     */
+    public static final String PIG_TEZ_SESSION_REUSE = "pig.tez.session.reuse";
+
+    /**
+     * This key is used to configure the interval of dag status report in seconds. Default is 20
+     */
+    public static final String PIG_TEZ_DAG_STATUS_REPORT_INTERVAL = "pig.tez.dag.status.report.interval";
+
+
+
+    // Deprecated settings of Pig 0.13
+
+    /**
+     * @deprecated use {@link #PIG_OPT_FETCH} instead. Will be removed in Pig 0.16
+     */
+    @Deprecated
+    public static final String OPT_FETCH = PIG_OPT_FETCH;
+
+    /**
+     * @deprecated use {@link #PIG_CACHEDBAG_MEMUSAGE} instead. Will be removed in Pig 0.16
+     */
+    @Deprecated
+    public static final String PROP_CACHEDBAG_MEMUSAGE = PIG_CACHEDBAG_MEMUSAGE;
+
+    /**
+     * @deprecated use {@link #PIG_EXEC_MAP_PARTAGG} instead. Will be removed in Pig 0.16
+     */
+    @Deprecated
+    public static final String PROP_EXEC_MAP_PARTAGG = PIG_EXEC_MAP_PARTAGG;
+
+    /**
+     * @deprecated use {@link #PIG_EXEC_MAP_PARTAGG_MINREDUCTION} instead. Will be removed in Pig 0.16
+     */
+    @Deprecated
+    public static final String PARTAGG_MINREDUCTION = PIG_EXEC_MAP_PARTAGG_MINREDUCTION;
+
+    /**
+     * @deprecated use {@link #PROP_NO_COMBINER1} instead. Will be removed in Pig 0.16
+     */
+    @Deprecated
+    public static final String PROP_NO_COMBINER = PIG_EXEC_NO_COMBINER;
+
+    @Deprecated
+    public static final String SHOULD_USE_SCHEMA_TUPLE = PIG_SCHEMA_TUPLE_ENABLED;
+
+    @Deprecated
+    public static final String SCHEMA_TUPLE_SHOULD_USE_IN_UDF = PIG_SCHEMA_TUPLE_USE_IN_UDF;
+
+    @Deprecated
+    public static final String SCHEMA_TUPLE_SHOULD_USE_IN_FOREACH = PIG_SCHEMA_TUPLE_USE_IN_FOREACH;
+
+    @Deprecated
+    public static final String SCHEMA_TUPLE_SHOULD_USE_IN_FRJOIN = PIG_SCHEMA_TUPLE_USE_IN_FRJOIN;
+
+    @Deprecated
+    public static final String SCHEMA_TUPLE_SHOULD_USE_IN_MERGEJOIN = PIG_SCHEMA_TUPLE_USE_IN_MERGEJOIN;
+
+    @Deprecated
+    public static final String SCHEMA_TUPLE_SHOULD_ALLOW_FORCE = PIG_SCHEMA_TUPLE_ALLOW_FORCE;
+
+    /**
+     * @deprecated use {@link #PIG_SCRIPT_INFO_ENABLED} instead. Will be removed in Pig 0.16
+     */
+    @Deprecated
+    public static final String INSERT_ENABLED = PIG_SCRIPT_INFO_ENABLED;
+
+    /**
+     * @deprecated use {@link #PIG_SCRIPT_MAX_SIZE} instead. Will be removed in Pig 0.16
+     */
+    @Deprecated
+    public static final String MAX_SCRIPT_SIZE = PIG_SCRIPT_MAX_SIZE;
+
 }

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/fetch/FetchOptimizer.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/fetch/FetchOptimizer.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/fetch/FetchOptimizer.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/fetch/FetchOptimizer.java Sun Nov  9 23:04:59 2014
@@ -81,7 +81,7 @@ public class FetchOptimizer {
      */
     public static boolean isFetchEnabled(PigContext pc) {
         return "true".equalsIgnoreCase(
-                pc.getProperties().getProperty(PigConfiguration.OPT_FETCH, "true"));
+                pc.getProperties().getProperty(PigConfiguration.PIG_OPT_FETCH, "true"));
     }
 
     /**

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=1637758&r1=1637757&r2=1637758&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 Sun Nov  9 23:04:59 2014
@@ -17,6 +17,9 @@
  */
 package org.apache.pig.backend.hadoop.executionengine.mapReduceLayer;
 
+import static org.apache.pig.PigConfiguration.PIG_EXEC_REDUCER_ESTIMATOR;
+import static org.apache.pig.PigConfiguration.PIG_EXEC_REDUCER_ESTIMATOR_CONSTRUCTOR_ARG_KEY;
+
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
@@ -161,9 +164,6 @@ public class JobControlCompiler{
 
     public static final String END_OF_INP_IN_MAP = "pig.invoke.close.in.map";
 
-    private static final String REDUCER_ESTIMATOR_KEY = "pig.exec.reducer.estimator";
-    private static final String REDUCER_ESTIMATOR_ARG_KEY =  "pig.exec.reducer.estimator.arg";
-
     public static final String PIG_MAP_COUNTER = "pig.counters.counter_";
     public static final String PIG_MAP_RANK_NAME = "pig.rank_";
     public static final String PIG_MAP_SEPARATOR = "_";
@@ -503,7 +503,7 @@ public class JobControlCompiler{
         Path tmpLocation = null;
 
         // add settings for pig statistics
-        String setScriptProp = conf.get(PigConfiguration.INSERT_ENABLED, "true");
+        String setScriptProp = conf.get(PigConfiguration.PIG_SCRIPT_INFO_ENABLED, "true");
         if (setScriptProp.equalsIgnoreCase("true")) {
             MRScriptState ss = MRScriptState.get();
             ss.addSettingsToConf(mro, conf);
@@ -1104,10 +1104,10 @@ public class JobControlCompiler{
             MapReduceOper mapReducerOper) throws IOException {
         Configuration conf = job.getConfiguration();
 
-        PigReducerEstimator estimator = conf.get(REDUCER_ESTIMATOR_KEY) == null ?
+        PigReducerEstimator estimator = conf.get(PIG_EXEC_REDUCER_ESTIMATOR) == null ?
                 new InputSizeReducerEstimator() :
                     PigContext.instantiateObjectFromParams(conf,
-                            REDUCER_ESTIMATOR_KEY, REDUCER_ESTIMATOR_ARG_KEY, PigReducerEstimator.class);
+                            PIG_EXEC_REDUCER_ESTIMATOR, PIG_EXEC_REDUCER_ESTIMATOR_CONSTRUCTOR_ARG_KEY, PigReducerEstimator.class);
 
                 log.info("Using reducer estimator: " + estimator.getClass().getName());
                 int numberOfReducers = estimator.estimateNumberOfReducers(job, mapReducerOper);

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java Sun Nov  9 23:04:59 2014
@@ -638,10 +638,10 @@ public class MapReduceLauncher extends L
                 pc.getProperties().getProperty(
                         "last.input.chunksize", JoinPackager.DEFAULT_CHUNK_SIZE);
 
-        String prop = pc.getProperties().getProperty(PigConfiguration.PROP_NO_COMBINER);
+        String prop = pc.getProperties().getProperty(PigConfiguration.PIG_EXEC_NO_COMBINER);
         if (!pc.inIllustrator && !("true".equals(prop)))  {
             boolean doMapAgg =
-                    Boolean.valueOf(pc.getProperties().getProperty(PigConfiguration.PROP_EXEC_MAP_PARTAGG,"false"));
+                    Boolean.valueOf(pc.getProperties().getProperty(PigConfiguration.PIG_EXEC_MAP_PARTAGG,"false"));
             CombinerOptimizer co = new CombinerOptimizer(plan, doMapAgg);
             co.visit();
             //display the warning message(s) from the CombinerOptimizer
@@ -687,7 +687,7 @@ public class MapReduceLauncher extends L
         fRem.visit();
 
         boolean isMultiQuery =
-            Boolean.valueOf(pc.getProperties().getProperty(PigConfiguration.OPT_MULTIQUERY, "true"));
+            Boolean.valueOf(pc.getProperties().getProperty(PigConfiguration.PIG_OPT_MULTIQUERY, "true"));
 
         if (isMultiQuery) {
             // reduces the number of MROpers in the MR plan generated

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigRecordReader.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigRecordReader.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigRecordReader.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigRecordReader.java Sun Nov  9 23:04:59 2014
@@ -17,8 +17,8 @@
  */
 package org.apache.pig.backend.hadoop.executionengine.mapReduceLayer;
 
-import static org.apache.pig.PigConfiguration.TIME_UDFS;
-import static org.apache.pig.PigConfiguration.TIME_UDFS_FREQUENCY;
+import static org.apache.pig.PigConfiguration.PIG_UDF_PROFILE;
+import static org.apache.pig.PigConfiguration.PIG_UDF_PROFILE_FREQUENCY;
 import static org.apache.pig.PigConstants.TIME_UDFS_ELAPSED_TIME_COUNTER;
 
 import java.io.IOException;
@@ -119,10 +119,10 @@ public class PigRecordReader extends Rec
         idx = 0;
         this.limit = limit;
         initNextRecordReader();
-        doTiming = inputSpecificConf.getBoolean(TIME_UDFS, false);
+        doTiming = inputSpecificConf.getBoolean(PIG_UDF_PROFILE, false);
         if (doTiming) {
             counterGroup = loadFunc.toString();
-            timingFrequency = inputSpecificConf.getLong(TIME_UDFS_FREQUENCY, 100L);
+            timingFrequency = inputSpecificConf.getLong(PIG_UDF_PROFILE_FREQUENCY, 100L);
         }
     }
 

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java Sun Nov  9 23:04:59 2014
@@ -18,10 +18,10 @@
 
 package org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators;
 
-import static org.apache.pig.PigConfiguration.TIME_UDFS;
-import static org.apache.pig.PigConfiguration.TIME_UDFS_FREQUENCY;
-import static org.apache.pig.PigConstants.TIME_UDFS_INVOCATION_COUNTER;
+import static org.apache.pig.PigConfiguration.PIG_UDF_PROFILE;
+import static org.apache.pig.PigConfiguration.PIG_UDF_PROFILE_FREQUENCY;
 import static org.apache.pig.PigConstants.TIME_UDFS_ELAPSED_TIME_COUNTER;
+import static org.apache.pig.PigConstants.TIME_UDFS_INVOCATION_COUNTER;
 
 import java.io.IOException;
 import java.io.ObjectInputStream;
@@ -37,7 +37,6 @@ import org.apache.pig.Algebraic;
 import org.apache.pig.EvalFunc;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.PigException;
-import org.apache.pig.PigWarning;
 import org.apache.pig.TerminatingAccumulator;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
@@ -154,10 +153,10 @@ public class POUserFunc extends Expressi
             func.setPigLogger(pigLogger);
             Configuration jobConf = UDFContext.getUDFContext().getJobConf();
             if (jobConf != null) {
-                doTiming = jobConf.getBoolean(TIME_UDFS, false);
+                doTiming = jobConf.getBoolean(PIG_UDF_PROFILE, false);
                 if (doTiming) {
                     counterGroup = funcSpec.toString();
-                    timingFrequency = jobConf.getLong(TIME_UDFS_FREQUENCY, 100L);
+                    timingFrequency = jobConf.getLong(PIG_UDF_PROFILE_FREQUENCY, 100L);
                 }
             }
             // We initialize here instead of instantiateFunc because this is called
@@ -530,7 +529,7 @@ public class POUserFunc extends Expressi
     public FuncSpec getFuncSpec() {
         return funcSpec;
     }
-    
+
     public void setFuncSpec(FuncSpec funcSpec) {
         this.funcSpec = funcSpec;
         instantiateFunc(funcSpec);

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPartialAgg.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPartialAgg.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPartialAgg.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPartialAgg.java Sun Nov  9 23:04:59 2014
@@ -150,7 +150,7 @@ public class POPartialAgg extends Physic
                 percentUsage = Float.parseFloat(usage);
             }
             minOutputReduction = PigMapReduce.sJobConfInternal.get().getInt(
-                    PigConfiguration.PARTAGG_MINREDUCTION, DEFAULT_MIN_REDUCTION);
+                    PigConfiguration.PIG_EXEC_MAP_PARTAGG_MINREDUCTION, DEFAULT_MIN_REDUCTION);
             if (minOutputReduction <= 0) {
                 LOG.info("Specified reduction is < 0 (" + minOutputReduction + "). Using default " +
                         DEFAULT_MIN_REDUCTION);

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezJob.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezJob.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezJob.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezJob.java Sun Nov  9 23:04:59 2014
@@ -74,7 +74,7 @@ public class TezJob implements Runnable 
         this.conf = conf;
         this.dag = dag;
         this.requestAMResources = requestAMResources;
-        this.reuseSession = conf.getBoolean(PigConfiguration.TEZ_SESSION_REUSE, true);
+        this.reuseSession = conf.getBoolean(PigConfiguration.PIG_TEZ_SESSION_REUSE, true);
         this.statusGetOpts = EnumSet.of(StatusGetOpts.GET_COUNTERS);
         tezJobConf = new TezJobConfig(estimatedTotalParallelism);
     }
@@ -173,7 +173,7 @@ public class TezJob implements Runnable 
 
         timer = new Timer();
         timer.schedule(new DAGStatusReporter(), 1000, conf.getLong(
-                PigConfiguration.TEZ_DAG_STATUS_REPORT_INTERVAL, 20) * 1000);
+                PigConfiguration.PIG_TEZ_DAG_STATUS_REPORT_INTERVAL, 20) * 1000);
 
         while (true) {
             try {

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java Sun Nov  9 23:04:59 2014
@@ -109,7 +109,7 @@ public class TezLauncher extends Launche
             pc.getProperties().setProperty("tez.ignore.lib.uris", "true");
         }
         Configuration conf = ConfigurationUtil.toConfiguration(pc.getProperties(), true);
-        if (pc.defaultParallel == -1 && !conf.getBoolean(PigConfiguration.TEZ_AUTO_PARALLELISM, true)) {
+        if (pc.defaultParallel == -1 && !conf.getBoolean(PigConfiguration.PIG_TEZ_AUTO_PARALLELISM, true)) {
             pc.defaultParallel = 1;
         }
         aggregateWarning = conf.getBoolean("aggregate.warning", false);
@@ -379,10 +379,10 @@ public class TezLauncher extends Launche
         filter.visit();
 
         // Run CombinerOptimizer on Tez plan
-        boolean nocombiner = conf.getBoolean(PigConfiguration.PROP_NO_COMBINER, false);
+        boolean nocombiner = conf.getBoolean(PigConfiguration.PIG_EXEC_NO_COMBINER, false);
         if (!pc.inIllustrator && !nocombiner)  {
             boolean doMapAgg = Boolean.parseBoolean(pc.getProperties().getProperty(
-                    PigConfiguration.PROP_EXEC_MAP_PARTAGG, "false"));
+                    PigConfiguration.PIG_EXEC_MAP_PARTAGG, "false"));
             CombinerOptimizer co = new CombinerOptimizer(tezPlan, doMapAgg);
             co.visit();
             co.getMessageCollector().logMessages(MessageType.Warning, aggregateWarning, log);
@@ -396,7 +396,7 @@ public class TezLauncher extends Launche
             skOptimizer.visit();
         }
 
-        boolean isMultiQuery = conf.getBoolean(PigConfiguration.OPT_MULTIQUERY, true);
+        boolean isMultiQuery = conf.getBoolean(PigConfiguration.PIG_OPT_MULTIQUERY, true);
         if (isMultiQuery) {
             // reduces the number of TezOpers in the Tez plan generated
             // by multi-query (multi-store) script.
@@ -405,14 +405,14 @@ public class TezLauncher extends Launche
         }
 
         // Run AccumulatorOptimizer on Tez plan
-        boolean isAccum = conf.getBoolean(PigConfiguration.OPT_ACCUMULATOR, true);
+        boolean isAccum = conf.getBoolean(PigConfiguration.PIG_OPT_ACCUMULATOR, true);
         if (isAccum) {
             AccumulatorOptimizer accum = new AccumulatorOptimizer(tezPlan);
             accum.visit();
         }
 
         // Use VertexGroup in Tez
-        boolean isUnionOpt = conf.getBoolean(PigConfiguration.TEZ_OPT_UNION, true);
+        boolean isUnionOpt = conf.getBoolean(PigConfiguration.PIG_TEZ_OPT_UNION, true);
         if (isUnionOpt) {
             UnionOptimizer uo = new UnionOptimizer(tezPlan);
             uo.visit();

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/TezCompiler.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/TezCompiler.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/TezCompiler.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/TezCompiler.java Sun Nov  9 23:04:59 2014
@@ -1467,12 +1467,12 @@ public class TezCompiler extends PhyPlan
             POLocalRearrangeTez lrTezSample = localRearrangeFactory.create(LocalRearrangeType.NULL);
 
             int sampleRate = POPoissonSample.DEFAULT_SAMPLE_RATE;
-            if (pigProperties.containsKey(PigConfiguration.SAMPLE_RATE)) {
-                sampleRate = Integer.valueOf(pigProperties.getProperty(PigConfiguration.SAMPLE_RATE));
+            if (pigProperties.containsKey(PigConfiguration.PIG_POISSON_SAMPLER_SAMPLE_RATE)) {
+                sampleRate = Integer.valueOf(pigProperties.getProperty(PigConfiguration.PIG_POISSON_SAMPLER_SAMPLE_RATE));
             }
             float heapPerc =  PartitionSkewedKeys.DEFAULT_PERCENT_MEMUSAGE;
-            if (pigProperties.containsKey(PigConfiguration.PERC_MEM_AVAIL)) {
-                heapPerc = Float.valueOf(pigProperties.getProperty(PigConfiguration.PERC_MEM_AVAIL));
+            if (pigProperties.containsKey(PigConfiguration.PIG_SKEWEDJOIN_REDUCE_MEMUSAGE)) {
+                heapPerc = Float.valueOf(pigProperties.getProperty(PigConfiguration.PIG_SKEWEDJOIN_REDUCE_MEMUSAGE));
             }
             POPoissonSample poSample = new POPoissonSample(new OperatorKey(scope,nig.getNextNodeId(scope)),
                     -1, sampleRate, heapPerc);

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/optimizer/ParallelismSetter.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/optimizer/ParallelismSetter.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/optimizer/ParallelismSetter.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/optimizer/ParallelismSetter.java Sun Nov  9 23:04:59 2014
@@ -55,11 +55,11 @@ public class ParallelismSetter extends T
         super(plan, new DependencyOrderWalker<TezOperator, TezOperPlan>(plan));
         this.pc = pigContext;
         this.conf = ConfigurationUtil.toConfiguration(pc.getProperties());
-        this.autoParallelismEnabled = conf.getBoolean(PigConfiguration.TEZ_AUTO_PARALLELISM, true);
+        this.autoParallelismEnabled = conf.getBoolean(PigConfiguration.PIG_TEZ_AUTO_PARALLELISM, true);
         try {
-            this.estimator = conf.get(PigConfiguration.REDUCER_ESTIMATOR_KEY) == null ? new TezOperDependencyParallelismEstimator()
+            this.estimator = conf.get(PigConfiguration.PIG_EXEC_REDUCER_ESTIMATOR) == null ? new TezOperDependencyParallelismEstimator()
             : PigContext.instantiateObjectFromParams(conf,
-                    PigConfiguration.REDUCER_ESTIMATOR_KEY, PigConfiguration.REDUCER_ESTIMATOR_ARG_KEY,
+                    PigConfiguration.PIG_EXEC_REDUCER_ESTIMATOR, PigConfiguration.PIG_EXEC_REDUCER_ESTIMATOR_CONSTRUCTOR_ARG_KEY,
                     TezParallelismEstimator.class);
             this.estimator.setPigContext(pc);
 

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/AccumulatorOptimizerUtil.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/AccumulatorOptimizerUtil.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/AccumulatorOptimizerUtil.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/AccumulatorOptimizerUtil.java Sun Nov  9 23:04:59 2014
@@ -32,7 +32,7 @@ public class AccumulatorOptimizerUtil {
     public static int getAccumulativeBatchSize() {
         int batchSize = 20000;
         if (PigMapReduce.sJobConfInternal.get() != null) {
-            String size = PigMapReduce.sJobConfInternal.get().get(PigConfiguration.ACCUMULATIVE_BATCHSIZE);
+            String size = PigMapReduce.sJobConfInternal.get().get(PigConfiguration.PIG_ACCUMULATIVE_BATCHSIZE);
             if (size != null) {
                 batchSize = Integer.parseInt(size);
             }

Modified: pig/trunk/src/org/apache/pig/data/SchemaTupleBackend.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/data/SchemaTupleBackend.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/data/SchemaTupleBackend.java (original)
+++ pig/trunk/src/org/apache/pig/data/SchemaTupleBackend.java Sun Nov  9 23:04:59 2014
@@ -17,7 +17,7 @@
  */
 package org.apache.pig.data;
 
-import static org.apache.pig.PigConfiguration.SHOULD_USE_SCHEMA_TUPLE;
+import static org.apache.pig.PigConfiguration.PIG_SCHEMA_TUPLE_ENABLED;
 import static org.apache.pig.PigConstants.SCHEMA_TUPLE_ON_BY_DEFAULT;
 
 import java.io.File;
@@ -33,7 +33,6 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.pig.ExecType;
 import org.apache.pig.PigConstants;
 import org.apache.pig.data.SchemaTupleClassGenerator.GenContext;
 import org.apache.pig.data.utils.StructuresHelper.SchemaKey;
@@ -151,8 +150,8 @@ public class SchemaTupleBackend {
             return;
         }
         // Step one is to see if there are any classes in the distributed cache
-        if (!jConf.getBoolean(SHOULD_USE_SCHEMA_TUPLE, SCHEMA_TUPLE_ON_BY_DEFAULT)) {
-            LOG.info("Key [" + SHOULD_USE_SCHEMA_TUPLE +"] was not set... will not generate code.");
+        if (!jConf.getBoolean(PIG_SCHEMA_TUPLE_ENABLED, SCHEMA_TUPLE_ON_BY_DEFAULT)) {
+            LOG.info("Key [" + PIG_SCHEMA_TUPLE_ENABLED +"] was not set... will not generate code.");
             return;
         }
         // Step two is to copy everything from the distributed cache if we are in distributed mode
@@ -189,7 +188,7 @@ public class SchemaTupleBackend {
             try {
                 fin = new FileInputStream(src);
                 fos = new FileOutputStream(new File(codeDir, s));
-    
+
                 fin.getChannel().transferTo(0, src.length(), fos.getChannel());
                 LOG.info("Successfully copied file to local directory.");
             } finally {

Modified: pig/trunk/src/org/apache/pig/data/SchemaTupleClassGenerator.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/data/SchemaTupleClassGenerator.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/data/SchemaTupleClassGenerator.java (original)
+++ pig/trunk/src/org/apache/pig/data/SchemaTupleClassGenerator.java Sun Nov  9 23:04:59 2014
@@ -61,27 +61,27 @@ public class SchemaTupleClassGenerator {
          * This context is used in UDF code. Currently, this is only used for
          * the inputs to UDF's.
          */
-        UDF (PigConfiguration.SCHEMA_TUPLE_SHOULD_USE_IN_UDF, true, GenerateUdf.class),
+        UDF (PigConfiguration.PIG_SCHEMA_TUPLE_USE_IN_UDF, true, GenerateUdf.class),
         /**
          * This context is for POForEach. This will use the expected output of a ForEach
          * to return a typed Tuple.
          */
-        FOREACH (PigConfiguration.SCHEMA_TUPLE_SHOULD_USE_IN_FOREACH, true, GenerateForeach.class),
+        FOREACH (PigConfiguration.PIG_SCHEMA_TUPLE_USE_IN_FOREACH, true, GenerateForeach.class),
         /**
          * This context controls whether or not SchemaTuples will be used in FR joins.
          * Currently, they will be used in the HashMap that FR Joins construct.
          */
-        FR_JOIN (PigConfiguration.SCHEMA_TUPLE_SHOULD_USE_IN_FRJOIN, true, GenerateFrJoin.class),
+        FR_JOIN (PigConfiguration.PIG_SCHEMA_TUPLE_USE_IN_FRJOIN, true, GenerateFrJoin.class),
         /**
          * This context controls whether or not SchemaTuples will be used in merge joins.
          */
-        MERGE_JOIN (PigConfiguration.SCHEMA_TUPLE_SHOULD_USE_IN_MERGEJOIN, true, GenerateMergeJoin.class),
+        MERGE_JOIN (PigConfiguration.PIG_SCHEMA_TUPLE_USE_IN_MERGEJOIN, true, GenerateMergeJoin.class),
         /**
          * All registered Schemas will also be registered in one additional context.
          * This context will allow users to "force" the load of a SchemaTupleFactory
          * if one is present in any context.
          */
-        FORCE_LOAD (PigConfiguration.SCHEMA_TUPLE_SHOULD_ALLOW_FORCE, true, GenerateForceLoad.class);
+        FORCE_LOAD (PigConfiguration.PIG_SCHEMA_TUPLE_ALLOW_FORCE, true, GenerateForceLoad.class);
 
         /**
          * These annotations are used to mark a given SchemaTuple with
@@ -226,7 +226,7 @@ public class SchemaTupleClassGenerator {
      */
     //TODO in the future, we can use ASM to generate the bytecode directly.
     private static void compileCodeString(String className, String generatedCodeString, File codeDir) {
-        JavaCompilerHelper compiler = new JavaCompilerHelper(); 
+        JavaCompilerHelper compiler = new JavaCompilerHelper();
         String tempDir = codeDir.getAbsolutePath();
         compiler.addToClassPath(tempDir);
         LOG.debug("Compiling SchemaTuple code with classpath: " + compiler.getClassPath());
@@ -242,12 +242,14 @@ public class SchemaTupleClassGenerator {
             this.id = id;
         }
 
+        @Override
         public void prepare() {
             add("@Override");
             add("protected int generatedCodeCompareToSpecific(SchemaTuple_"+id+" t) {");
             add("    int i = 0;");
         }
 
+        @Override
         public void process(int fieldNum, Schema.FieldSchema fs) {
             add("    i = compare(checkIfNull_" + fieldNum + "(), getPos_"
                     + fieldNum + "(), t.checkIfNull_" + fieldNum + "(), t.getPos_"
@@ -257,6 +259,7 @@ public class SchemaTupleClassGenerator {
             add("    }");
         }
 
+        @Override
         public void end() {
             add("    return i;");
             add("}");
@@ -271,6 +274,7 @@ public class SchemaTupleClassGenerator {
             this.id = id;
         }
 
+        @Override
         public void prepare() {
             add("@Override");
             add("protected int generatedCodeCompareTo(SchemaTuple t, boolean checkType) {");
@@ -282,6 +286,7 @@ public class SchemaTupleClassGenerator {
         boolean compIsNull = false;
         boolean compByte = false;
 
+        @Override
         public void process(int fieldNum, Schema.FieldSchema fs) {
             add("        i = compareWithElementAtPos(checkIfNull_" + fieldNum + "(), getPos_" + fieldNum + "(), t, " + fieldNum + ");");
             add("        if (i != 0) {");
@@ -289,6 +294,7 @@ public class SchemaTupleClassGenerator {
             add("        }");
         }
 
+        @Override
         public void end() {
             add("    return 0;");
             add("}");
@@ -296,16 +302,19 @@ public class SchemaTupleClassGenerator {
     }
 
     static class HashCode extends TypeInFunctionStringOut {
+        @Override
         public void prepare() {
             add("@Override");
             add("public int generatedCodeHashCode() {");
             add("    int h = 17;");
         }
 
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             add("    h = hashCodePiece(h, getPos_" + fieldPos + "(), checkIfNull_" + fieldPos + "());");
         }
 
+        @Override
         public void end() {
             add("    return h;");
             add("}");
@@ -323,6 +332,7 @@ public class SchemaTupleClassGenerator {
         private int booleans = 0;
         private File codeDir;
 
+        @Override
         public void prepare() {
             String s;
             try {
@@ -333,6 +343,7 @@ public class SchemaTupleClassGenerator {
             add("private static Schema schema = staticSchemaGen(\"" + s + "\");");
         }
 
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             if (!isTuple()) {
                 if (isPrimitive() && (primitives++ % 8 == 0)) {
@@ -385,6 +396,7 @@ public class SchemaTupleClassGenerator {
         private int byteField = 0; //this is for setting booleans
         private int byteIncr = 0; //this is for counting the booleans we've encountered
 
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             if (!isTuple()) {
                 add("public void setPos_"+fieldPos+"("+typeName()+" v) {");
@@ -433,27 +445,32 @@ public class SchemaTupleClassGenerator {
     }
 
     static class ListSetString extends TypeInFunctionStringOut {
+        @Override
         public void prepare() {
             add("@Override");
             add("public void generatedCodeSetIterator(Iterator<Object> it) throws ExecException {");
         }
 
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             add("    setPos_"+fieldPos+"(unbox(it.next(), getDummy_"+fieldPos+"()));");
         }
 
+        @Override
         public void end() {
             add("}");
         }
     }
 
     static class GenericSetString extends TypeInFunctionStringOut {
+        @Override
         public void prepare() {
             add("@Override");
             add("public void generatedCodeSetField(int fieldNum, Object val) throws ExecException {");
             add("    switch (fieldNum) {");
         }
 
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             add("    case ("+fieldPos+"):");
             add("        if (val == null) {");
@@ -464,6 +481,7 @@ public class SchemaTupleClassGenerator {
             add("        break;");
         }
 
+        @Override
         public void end() {
             add("    default:");
             add("        throw new ExecException(\"Invalid index given to set: \" + fieldNum);");
@@ -473,16 +491,19 @@ public class SchemaTupleClassGenerator {
     }
 
     static class GenericGetString extends TypeInFunctionStringOut {
+        @Override
         public void prepare() {
             add("@Override");
             add("public Object generatedCodeGetField(int fieldNum) throws ExecException {");
             add("    switch (fieldNum) {");
         }
 
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             add("    case ("+fieldPos+"): return checkIfNull_"+fieldPos+"() ? null : box(getPos_"+fieldPos+"());");
         }
 
+        @Override
         public void end() {
             add("    default: throw new ExecException(\"Invalid index given to get: \" + fieldNum);");
             add("    }");
@@ -491,16 +512,19 @@ public class SchemaTupleClassGenerator {
     }
 
     static class GeneralIsNullString extends TypeInFunctionStringOut {
+        @Override
         public void prepare() {
             add("@Override");
             add("public boolean isGeneratedCodeFieldNull(int fieldNum) throws ExecException {");
             add("    switch (fieldNum) {");
         }
 
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             add("    case ("+fieldPos+"): return checkIfNull_"+fieldPos+"();");
         }
 
+        @Override
         public void end() {
             add("    default: throw new ExecException(\"Invalid index given: \" + fieldNum);");
             add("    }");
@@ -512,6 +536,7 @@ public class SchemaTupleClassGenerator {
         private int nullByte = 0; //the byte_ val
         private int byteIncr = 0; //the mask we're on
 
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             add("public boolean checkIfNull_" + fieldPos + "() {");
             if (isPrimitive()) {
@@ -532,6 +557,7 @@ public class SchemaTupleClassGenerator {
         private int nullByte = 0; //the byte_ val
         private int byteIncr = 0; //the mask we're on
 
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             add("public void setNull_"+fieldPos+"(boolean b) {");
             if (isPrimitive()) {
@@ -554,11 +580,13 @@ public class SchemaTupleClassGenerator {
     static class SetEqualToSchemaTupleSpecificString extends TypeInFunctionStringOut {
         private int id;
 
+        @Override
         public void prepare() {
             add("@Override");
             add("protected SchemaTuple generatedCodeSetSpecific(SchemaTuple_"+id+" t) {");
         }
 
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             add("    if (t.checkIfNull_" + fieldPos + "()) {");
             add("        setNull_" + fieldPos + "(true);");
@@ -568,6 +596,7 @@ public class SchemaTupleClassGenerator {
             addBreak();
         }
 
+        @Override
         public void end() {
             add("    return this;");
             add("}");
@@ -586,6 +615,7 @@ public class SchemaTupleClassGenerator {
             this.id = id;
         }
 
+        @Override
         public void prepare() {
             add("@Override");
             add("public boolean isSpecificSchemaTuple(Object o) {");
@@ -599,15 +629,18 @@ public class SchemaTupleClassGenerator {
     static class WriteNullsString extends TypeInFunctionStringOut {
         String s = "    boolean[] b = {\n";
 
+        @Override
         public void prepare() {
             add("@Override");
             add("protected boolean[] generatedCodeNullsArray() throws IOException {");
         }
 
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             s += "        checkIfNull_"+fieldPos+"(),\n";
         }
 
+        @Override
         public void end() {
             s = s.substring(0, s.length() - 2) + "\n    };";
             add(s);
@@ -626,11 +659,13 @@ public class SchemaTupleClassGenerator {
 
         private int booleans = 0;
 
+        @Override
         public void prepare() {
             add("@Override");
             add("protected void generatedCodeReadFields(DataInput in, boolean[] b) throws IOException {");
         }
 
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             if (isBoolean()) {
                 booleans++;
@@ -659,6 +694,7 @@ public class SchemaTupleClassGenerator {
             }
         }
 
+        @Override
         public void end() {
             if (booleans > 0) {
                 int i = 0;
@@ -679,6 +715,7 @@ public class SchemaTupleClassGenerator {
 
 
     static class WriteString extends TypeInFunctionStringOut {
+        @Override
         public void prepare() {
             add("@Override");
             add("protected void generatedCodeWriteElements(DataOutput out) throws IOException {");
@@ -686,6 +723,7 @@ public class SchemaTupleClassGenerator {
 
         private int booleans = 0;
 
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             if (isBoolean()) {
                 booleans++;
@@ -697,6 +735,7 @@ public class SchemaTupleClassGenerator {
             }
         }
 
+        @Override
         public void end() {
             if (booleans > 0) {
                 int i = 0;
@@ -716,6 +755,7 @@ public class SchemaTupleClassGenerator {
 
         String s = "    return SizeUtil.roundToEight(";
 
+        @Override
         public void prepare() {
             add("@Override");
             add("public long getGeneratedCodeMemorySize() {");
@@ -725,6 +765,7 @@ public class SchemaTupleClassGenerator {
         private int primitives = 0;
 
         //TODO a null array or object variable still takes up space for the pointer, yes?
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             if (isInt() || isFloat()) {
                 size += 4;
@@ -757,6 +798,7 @@ public class SchemaTupleClassGenerator {
             }
         }
 
+        @Override
         public void end() {
             s += size + ");";
             add(s);
@@ -766,6 +808,7 @@ public class SchemaTupleClassGenerator {
     }
 
     static class GetDummyString extends TypeInFunctionStringOut {
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             add("public "+typeName()+" getDummy_"+fieldPos+"() {");
             switch (fs.type) {
@@ -795,6 +838,7 @@ public class SchemaTupleClassGenerator {
         private int booleanByte = 0;
         private int booleans;
 
+        @Override
         public void process(int fieldPos, Schema.FieldSchema fs) {
             if (!isTuple()) {
                 add("public "+typeName()+" getPos_"+fieldPos+"() {");
@@ -823,6 +867,7 @@ public class SchemaTupleClassGenerator {
     static class GetSchemaTupleIdentifierString extends TypeInFunctionStringOut {
         private int id;
 
+        @Override
         public void end() {
             add("@Override");
             add("public int getSchemaTupleIdentifier() {");
@@ -839,10 +884,12 @@ public class SchemaTupleClassGenerator {
     static class SchemaSizeString extends TypeInFunctionStringOut {
         int i = 0;
 
+        @Override
         public void process(int fieldNum, Schema.FieldSchema fS) {
             i++;
         }
 
+        @Override
         public void end() {
             add("@Override");
             add("protected int schemaSize() {");
@@ -855,10 +902,12 @@ public class SchemaTupleClassGenerator {
     static class SizeString extends TypeInFunctionStringOut {
         int i = 0;
 
+        @Override
         public void process(int fieldNum, Schema.FieldSchema fS) {
             i++;
         }
 
+        @Override
         public void end() {
             add("@Override");
             add("protected int generatedCodeSize() {");
@@ -873,16 +922,19 @@ public class SchemaTupleClassGenerator {
     }
 
     static class GetTypeString extends TypeInFunctionStringOut {
+        @Override
         public void prepare() {
             add("@Override");
             add("public byte getGeneratedCodeFieldType(int fieldNum) throws ExecException {");
             add("    switch (fieldNum) {");
         }
 
+        @Override
         public void process(int fieldNum, Schema.FieldSchema fs) {
             add("    case ("+fieldNum+"): return "+fs.type+";");
         }
 
+        @Override
         public void end() {
             add("    default: throw new ExecException(\"Invalid index given: \" + fieldNum);");
             add("    }");
@@ -898,6 +950,7 @@ public class SchemaTupleClassGenerator {
             this.id = id;
         }
 
+        @Override
         public void prepare() {
             add("@Override");
             add("protected SchemaTuple generatedCodeSet(SchemaTuple t, boolean checkClass) throws ExecException {");
@@ -913,6 +966,7 @@ public class SchemaTupleClassGenerator {
             addBreak();
         }
 
+        @Override
         public void process(int fieldNum, Schema.FieldSchema fs) {
             add("    if ("+fs.type+" != theirFS.get("+fieldNum+").type) {");
             add("        throw new ExecException(\"Given SchemaTuple does not match current in field " + fieldNum + ". Expected type: " + fs.type + ", found: \" + theirFS.get("+fieldNum+").type);");
@@ -929,6 +983,7 @@ public class SchemaTupleClassGenerator {
             addBreak();
         }
 
+        @Override
         public void end() {
             add("    return this;");
             add("}");
@@ -940,18 +995,21 @@ public class SchemaTupleClassGenerator {
             super(type);
         }
 
+        @Override
         public void prepare() {
             add("@Override");
             add("protected "+name()+" generatedCodeGet"+properName()+"(int fieldNum) throws ExecException {");
             add("    switch(fieldNum) {");
         }
 
+        @Override
         public void process(int fieldNum, Schema.FieldSchema fs) {
             if (fs.type==thisType()) {
                 add("    case ("+fieldNum+"): return returnUnlessNull(checkIfNull_"+fieldNum+"(), getPos_"+fieldNum+"());");
             }
         }
 
+        @Override
         public void end() {
             add("    default:");
             add("        return unbox"+properName()+"(getTypeAwareBase(fieldNum, \""+name()+"\"));");
@@ -979,17 +1037,20 @@ public class SchemaTupleClassGenerator {
             return proper(thisType());
         }
 
+        @Override
         public void prepare() {
             add("@Override");
             add("protected void generatedCodeSet"+properName()+"(int fieldNum, "+name()+" val) throws ExecException {");
             add("    switch(fieldNum) {");
         }
 
+        @Override
         public void process(int fieldNum, Schema.FieldSchema fs) {
             if (fs.type==thisType())
                 add("    case ("+fieldNum+"): setPos_"+fieldNum+"(val); break;");
         }
 
+        @Override
         public void end() {
             add("    default: setTypeAwareBase(fieldNum, val, \""+name()+"\");");
             add("    }");

Modified: pig/trunk/src/org/apache/pig/data/SchemaTupleFrontend.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/data/SchemaTupleFrontend.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/data/SchemaTupleFrontend.java (original)
+++ pig/trunk/src/org/apache/pig/data/SchemaTupleFrontend.java Sun Nov  9 23:04:59 2014
@@ -17,7 +17,7 @@
  */
 package org.apache.pig.data;
 
-import static org.apache.pig.PigConfiguration.SHOULD_USE_SCHEMA_TUPLE;
+import static org.apache.pig.PigConfiguration.PIG_SCHEMA_TUPLE_ENABLED;
 import static org.apache.pig.PigConstants.GENERATED_CLASSES_KEY;
 import static org.apache.pig.PigConstants.LOCAL_CODE_DIR;
 import static org.apache.pig.PigConstants.SCHEMA_TUPLE_ON_BY_DEFAULT;
@@ -177,8 +177,8 @@ public class SchemaTupleFrontend {
          */
         private boolean generateAll(Map<Pair<SchemaKey, Boolean>, Pair<Integer, Set<GenContext>>> schemasToGenerate) {
             boolean filesToShip = false;
-            if (!conf.getBoolean(SHOULD_USE_SCHEMA_TUPLE, SCHEMA_TUPLE_ON_BY_DEFAULT)) {
-                LOG.info("Key ["+SHOULD_USE_SCHEMA_TUPLE+"] is false, will not generate code.");
+            if (!conf.getBoolean(PIG_SCHEMA_TUPLE_ENABLED, SCHEMA_TUPLE_ON_BY_DEFAULT)) {
+                LOG.info("Key ["+PIG_SCHEMA_TUPLE_ENABLED+"] is false, will not generate code.");
                 return false;
             }
             LOG.info("Generating all registered Schemas.");

Modified: pig/trunk/src/org/apache/pig/impl/builtin/PoissonSampleLoader.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/impl/builtin/PoissonSampleLoader.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/impl/builtin/PoissonSampleLoader.java (original)
+++ pig/trunk/src/org/apache/pig/impl/builtin/PoissonSampleLoader.java Sun Nov  9 23:04:59 2014
@@ -172,8 +172,8 @@ public class PoissonSampleLoader extends
         newSample = null;
 
         Configuration conf = split.getConf();
-        sampleRate = conf.getInt(PigConfiguration.SAMPLE_RATE, DEFAULT_SAMPLE_RATE);
-        heapPerc = conf.getFloat(PigConfiguration.PERC_MEM_AVAIL,
+        sampleRate = conf.getInt(PigConfiguration.PIG_POISSON_SAMPLER_SAMPLE_RATE, DEFAULT_SAMPLE_RATE);
+        heapPerc = conf.getFloat(PigConfiguration.PIG_SKEWEDJOIN_REDUCE_MEMUSAGE,
                 PartitionSkewedKeys.DEFAULT_PERCENT_MEMUSAGE);
     }
 

Modified: pig/trunk/src/org/apache/pig/impl/util/PropertiesUtil.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/impl/util/PropertiesUtil.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/impl/util/PropertiesUtil.java (original)
+++ pig/trunk/src/org/apache/pig/impl/util/PropertiesUtil.java Sun Nov  9 23:04:59 2014
@@ -145,9 +145,9 @@ public class PropertiesUtil {
             properties.setProperty("stop.on.failure", ""+false);
         }
 
-        if (properties.getProperty(PigConfiguration.OPT_FETCH) == null) {
+        if (properties.getProperty(PigConfiguration.PIG_OPT_FETCH) == null) {
             //by default fetch optimization is on
-            properties.setProperty(PigConfiguration.OPT_FETCH, ""+true);
+            properties.setProperty(PigConfiguration.PIG_OPT_FETCH, ""+true);
         }
     }
     

Modified: pig/trunk/src/org/apache/pig/tools/pigstats/ScriptState.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/tools/pigstats/ScriptState.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/tools/pigstats/ScriptState.java (original)
+++ pig/trunk/src/org/apache/pig/tools/pigstats/ScriptState.java Sun Nov  9 23:04:59 2014
@@ -296,7 +296,7 @@ public abstract class ScriptState {
         // restrict the size of the script to be stored in job conf
         int maxScriptSize = 10240;
         if (pigContext != null) {
-            String prop = pigContext.getProperties().getProperty(PigConfiguration.MAX_SCRIPT_SIZE);
+            String prop = pigContext.getProperties().getProperty(PigConfiguration.PIG_SCRIPT_MAX_SIZE);
             if (prop != null) {
                 maxScriptSize = Integer.valueOf(prop);
             }

Modified: pig/trunk/test/org/apache/pig/data/TestSchemaTuple.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/data/TestSchemaTuple.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/data/TestSchemaTuple.java (original)
+++ pig/trunk/test/org/apache/pig/data/TestSchemaTuple.java Sun Nov  9 23:04:59 2014
@@ -78,7 +78,7 @@ public class TestSchemaTuple {
     @Before
     public void perTestInitialize() {
         props = new Properties();
-        props.setProperty(PigConfiguration.SHOULD_USE_SCHEMA_TUPLE, "true");
+        props.setProperty(PigConfiguration.PIG_SCHEMA_TUPLE_ENABLED, "true");
 
         conf = ConfigurationUtil.toConfiguration(props);
 

Modified: pig/trunk/test/org/apache/pig/test/TestAccumulator.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestAccumulator.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestAccumulator.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestAccumulator.java Sun Nov  9 23:04:59 2014
@@ -77,7 +77,7 @@ public class TestAccumulator {
     public void setUp() throws Exception {
         Util.resetStateForExecModeSwitch();
         // Drop stale configuration from previous test run
-        properties.remove(PigConfiguration.OPT_ACCUMULATOR);
+        properties.remove(PigConfiguration.PIG_OPT_ACCUMULATOR);
         pigServer = new PigServer(cluster.getExecType(), properties);
     }
 
@@ -632,7 +632,7 @@ public class TestAccumulator {
     @Test
     public void testAccumulatorOff() throws IOException{
         pigServer.getPigContext().getProperties().setProperty(
-                PigConfiguration.OPT_ACCUMULATOR, "false");
+                PigConfiguration.PIG_OPT_ACCUMULATOR, "false");
 
         pigServer.registerQuery("A = load '" + INPUT_FILE2 + "' as (id:int, fruit);");
         pigServer.registerQuery("B = group A by id;");
@@ -641,7 +641,7 @@ public class TestAccumulator {
 
         checkAccumulatorOff("C");
         pigServer.getPigContext().getProperties().setProperty(
-                PigConfiguration.OPT_ACCUMULATOR, "true");
+                PigConfiguration.PIG_OPT_ACCUMULATOR, "true");
 
     }
 

Modified: pig/trunk/test/org/apache/pig/test/TestAutoLocalMode.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestAutoLocalMode.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestAutoLocalMode.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestAutoLocalMode.java Sun Nov  9 23:04:59 2014
@@ -95,7 +95,7 @@ public class TestAutoLocalMode {
     @Before
     public void setUp() throws Exception{
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
-        pigServer.getPigContext().getExecutionEngine().setProperty(PigConfiguration.OPT_FETCH, "false");
+        pigServer.getPigContext().getExecutionEngine().setProperty(PigConfiguration.PIG_OPT_FETCH, "false");
         pigServer.getPigContext().getExecutionEngine().setProperty(PigConfiguration.PIG_AUTO_LOCAL_ENABLED, String.valueOf("true"));
         pigServer.getPigContext().getExecutionEngine().setProperty(PigConfiguration.PIG_AUTO_LOCAL_INPUT_MAXBYTES, "200");
 

Modified: pig/trunk/test/org/apache/pig/test/TestBatchAliases.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestBatchAliases.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestBatchAliases.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestBatchAliases.java Sun Nov  9 23:04:59 2014
@@ -38,7 +38,7 @@ public class TestBatchAliases {
 
     @Before
     public void setUp() throws Exception {
-        System.setProperty(PigConfiguration.OPT_MULTIQUERY, ""+true);
+        System.setProperty(PigConfiguration.PIG_OPT_MULTIQUERY, ""+true);
         myPig = new PigServer(ExecType.LOCAL, new Properties());
         deleteOutputFiles();
     }

Modified: pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java Sun Nov  9 23:04:59 2014
@@ -1431,7 +1431,7 @@ public class TestEvalPipeline2 {
     public void testNonStandardDataWithoutFetch() throws Exception{
         Assume.assumeTrue("Skip this test for TEZ. See PIG-3994", Util.isMapredExecType(cluster.getExecType()));
         Properties props = pigServer.getPigContext().getProperties();
-        props.setProperty(PigConfiguration.OPT_FETCH, "false");
+        props.setProperty(PigConfiguration.PIG_OPT_FETCH, "false");
         String[] input1 = {
                 "0",
         };
@@ -1448,7 +1448,7 @@ public class TestEvalPipeline2 {
             }
         }
         finally {
-            props.setProperty(PigConfiguration.OPT_FETCH, "true");
+            props.setProperty(PigConfiguration.PIG_OPT_FETCH, "true");
         }
     }
 

Modified: pig/trunk/test/org/apache/pig/test/TestFetch.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestFetch.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestFetch.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestFetch.java Sun Nov  9 23:04:59 2014
@@ -96,7 +96,7 @@ public class TestFetch {
     public void setUp() throws Exception{
         pigServer = new PigServer(ExecType.LOCAL, new Properties());
         // force direct fetch mode
-        pigServer.getPigContext().getProperties().setProperty(PigConfiguration.OPT_FETCH, "true");
+        pigServer.getPigContext().getProperties().setProperty(PigConfiguration.PIG_OPT_FETCH, "true");
     }
 
     @Test

Modified: pig/trunk/test/org/apache/pig/test/TestGrunt.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestGrunt.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestGrunt.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestGrunt.java Sun Nov  9 23:04:59 2014
@@ -67,7 +67,7 @@ public class TestGrunt {
 
     @BeforeClass
     public static void oneTimeSetup() throws Exception {
-        cluster.setProperty(PigConfiguration.OPT_MULTIQUERY,"true");
+        cluster.setProperty(PigConfiguration.PIG_OPT_MULTIQUERY,"true");
     }
 
     @AfterClass

Modified: pig/trunk/test/org/apache/pig/test/TestLoad.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestLoad.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestLoad.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestLoad.java Sun Nov  9 23:04:59 2014
@@ -306,7 +306,7 @@ public class TestLoad {
         boolean[] multiquery = {true, false};
 
         for (boolean b : multiquery) {
-            pc.getProperties().setProperty(PigConfiguration.OPT_MULTIQUERY, "" + b);
+            pc.getProperties().setProperty(PigConfiguration.PIG_OPT_MULTIQUERY, "" + b);
 
             DataStorage dfs = pc.getDfs();
             dfs.setActiveContainer(dfs.asContainer("/tmp"));

Modified: pig/trunk/test/org/apache/pig/test/TestMultiQuery.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestMultiQuery.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestMultiQuery.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestMultiQuery.java Sun Nov  9 23:04:59 2014
@@ -52,7 +52,7 @@ public class TestMultiQuery {
         Util.copyFromLocalToLocal(
                 "test/org/apache/pig/test/data/passwd2", "passwd2");
         Properties props = new Properties();
-        props.setProperty(PigConfiguration.OPT_MULTIQUERY, ""+true);
+        props.setProperty(PigConfiguration.PIG_OPT_MULTIQUERY, ""+true);
         myPig = new PigServer(ExecType.LOCAL, props);
     }
 

Modified: pig/trunk/test/org/apache/pig/test/TestMultiQueryBasic.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestMultiQueryBasic.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestMultiQueryBasic.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestMultiQueryBasic.java Sun Nov  9 23:04:59 2014
@@ -69,7 +69,7 @@ public class TestMultiQueryBasic {
         Util.copyFromLocalToLocal(
                 "test/org/apache/pig/test/data/passwd2", "passwd2");
         Properties props = new Properties();
-        props.setProperty(PigConfiguration.OPT_MULTIQUERY, ""+true);
+        props.setProperty(PigConfiguration.PIG_OPT_MULTIQUERY, ""+true);
         myPig = new PigServer(ExecType.LOCAL, props);
     }
 

Modified: pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java Sun Nov  9 23:04:59 2014
@@ -82,7 +82,7 @@ public class TestMultiQueryCompiler {
 
     @Before
     public void setUp() throws Exception {
-        cluster.setProperty(PigConfiguration.OPT_MULTIQUERY, ""+true);
+        cluster.setProperty(PigConfiguration.PIG_OPT_MULTIQUERY, ""+true);
         myPig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
         deleteOutputFiles();
     }

Modified: pig/trunk/test/org/apache/pig/test/TestMultiQueryLocal.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestMultiQueryLocal.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestMultiQueryLocal.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestMultiQueryLocal.java Sun Nov  9 23:04:59 2014
@@ -67,7 +67,7 @@ public class TestMultiQueryLocal {
     @Before
     public void setUp() throws Exception {
         PigContext context = new PigContext(ExecType.LOCAL, new Properties());
-        context.getProperties().setProperty(PigConfiguration.OPT_MULTIQUERY, ""+true);
+        context.getProperties().setProperty(PigConfiguration.PIG_OPT_MULTIQUERY, ""+true);
         myPig = new PigServer(context);
         myPig.getPigContext().getProperties().setProperty("pig.usenewlogicalplan", "false");
         myPig.getPigContext().getProperties().setProperty(PigConfiguration.PIG_TEMP_DIR, "build/test/tmp/");

Modified: pig/trunk/test/org/apache/pig/test/TestPOPartialAggPlan.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPOPartialAggPlan.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPOPartialAggPlan.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPOPartialAggPlan.java Sun Nov  9 23:04:59 2014
@@ -64,7 +64,7 @@ public class TestPOPartialAggPlan  {
     public void testMapAggPropFalse() throws Exception{
         //test with pig.exec.mapPartAgg set to false
         String query = getGByQuery();
-        pc.getProperties().setProperty(PigConfiguration.PROP_EXEC_MAP_PARTAGG, "false");
+        pc.getProperties().setProperty(PigConfiguration.PIG_EXEC_MAP_PARTAGG, "false");
         MROperPlan mrp = Util.buildMRPlan(query, pc);
         assertEquals(mrp.size(), 1);
 
@@ -75,7 +75,7 @@ public class TestPOPartialAggPlan  {
     public void testMapAggPropTrue() throws Exception{
         //test with pig.exec.mapPartAgg to true
         String query = getGByQuery();
-        pc.getProperties().setProperty(PigConfiguration.PROP_EXEC_MAP_PARTAGG, "true");
+        pc.getProperties().setProperty(PigConfiguration.PIG_EXEC_MAP_PARTAGG, "true");
         MROperPlan mrp = Util.buildMRPlan(query, pc);
         assertEquals(mrp.size(), 1);
 
@@ -102,7 +102,7 @@ public class TestPOPartialAggPlan  {
         String query = "l = load 'x' as (a,b,c);" +
                 "g = group l by a;" +
                 "f = foreach g generate group;";
-        pc.getProperties().setProperty(PigConfiguration.PROP_EXEC_MAP_PARTAGG, "true");
+        pc.getProperties().setProperty(PigConfiguration.PIG_EXEC_MAP_PARTAGG, "true");
         MROperPlan mrp = Util.buildMRPlan(query, pc);
         assertEquals(mrp.size(), 1);
 
@@ -115,7 +115,7 @@ public class TestPOPartialAggPlan  {
         String query = "l = load 'x' as (a,b,c);" +
                 "g = group l by a;" +
                 "f = foreach g generate group, COUNT(l.b), l.b;";
-        pc.getProperties().setProperty(PigConfiguration.PROP_EXEC_MAP_PARTAGG, "true");
+        pc.getProperties().setProperty(PigConfiguration.PIG_EXEC_MAP_PARTAGG, "true");
         MROperPlan mrp = Util.buildMRPlan(query, pc);
         assertEquals(mrp.size(), 1);
 

Modified: pig/trunk/test/org/apache/pig/test/TestPigRunner.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPigRunner.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPigRunner.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPigRunner.java Sun Nov  9 23:04:59 2014
@@ -175,7 +175,7 @@ public class TestPigRunner {
             Configuration conf = ConfigurationUtil.toConfiguration(stats.getPigProperties());
             assertTrue(conf.getBoolean("stop.on.failure", false));
             assertTrue(!conf.getBoolean("aggregate.warning", true));
-            assertTrue(!conf.getBoolean(PigConfiguration.OPT_MULTIQUERY, true));
+            assertTrue(!conf.getBoolean(PigConfiguration.PIG_OPT_MULTIQUERY, true));
             assertTrue(!conf.getBoolean("opt.fetch", true));
         } finally {
             new File(PIG_FILE).delete();
@@ -204,7 +204,7 @@ public class TestPigRunner {
             Configuration conf = ConfigurationUtil.toConfiguration(stats.getPigProperties());
             assertTrue(conf.getBoolean("stop.on.failure", false));
             assertTrue(!conf.getBoolean("aggregate.warning", true));
-            assertTrue(!conf.getBoolean(PigConfiguration.OPT_MULTIQUERY, true));
+            assertTrue(!conf.getBoolean(PigConfiguration.PIG_OPT_MULTIQUERY, true));
             assertTrue(conf.getBoolean("opt.fetch", true));
         } finally {
             new File(PIG_FILE).delete();

Modified: pig/trunk/test/org/apache/pig/test/TestPigServer.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPigServer.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPigServer.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPigServer.java Sun Nov  9 23:04:59 2014
@@ -867,7 +867,7 @@ public class TestPigServer {
 
         assertEquals("999", properties.getProperty("pig.exec.reducers.max"));
         assertEquals("true", properties.getProperty("aggregate.warning"));
-        assertEquals("true", properties.getProperty(PigConfiguration.OPT_MULTIQUERY));
+        assertEquals("true", properties.getProperty(PigConfiguration.PIG_OPT_MULTIQUERY));
         assertEquals("false", properties.getProperty("stop.on.failure"));
 
         //Test with properties file
@@ -877,7 +877,7 @@ public class TestPigServer {
 
         assertEquals("999", properties.getProperty("pig.exec.reducers.max"));
         assertEquals("true", properties.getProperty("aggregate.warning"));
-        assertEquals("true", properties.getProperty(PigConfiguration.OPT_MULTIQUERY));
+        assertEquals("true", properties.getProperty(PigConfiguration.PIG_OPT_MULTIQUERY));
         assertEquals("false", properties.getProperty("stop.on.failure"));
 
         PrintWriter out = new PrintWriter(new FileWriter(propertyFile));
@@ -889,7 +889,7 @@ public class TestPigServer {
 
         properties = PropertiesUtil.loadDefaultProperties();
         assertEquals("false", properties.getProperty("aggregate.warning"));
-        assertEquals("false", properties.getProperty(PigConfiguration.OPT_MULTIQUERY));
+        assertEquals("false", properties.getProperty(PigConfiguration.PIG_OPT_MULTIQUERY));
         assertEquals("true", properties.getProperty("stop.on.failure"));
 
         propertyFile.delete();

Modified: pig/trunk/test/org/apache/pig/test/TestPredeployedJar.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPredeployedJar.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPredeployedJar.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPredeployedJar.java Sun Nov  9 23:04:59 2014
@@ -54,7 +54,7 @@ public class TestPredeployedJar {
         logger.addAppender(appender);
         
         PigServer pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getConfiguration());
-        pigServer.getPigContext().getProperties().put(PigConfiguration.OPT_FETCH, "false");
+        pigServer.getPigContext().getProperties().put(PigConfiguration.PIG_OPT_FETCH, "false");
         String[] inputData = new String[] { "hello", "world" };
         Util.createInputFile(cluster, "a.txt", inputData);
         String guavaJar = JarManager.findContainingJar(com.google.common.collect.Multimaps.class);

Modified: pig/trunk/test/org/apache/pig/test/TestStore.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestStore.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestStore.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestStore.java Sun Nov  9 23:04:59 2014
@@ -902,7 +902,7 @@ public class TestStore {
     }
 
     private void checkStorePath(String orig, String expected, boolean isTmp) throws Exception {
-        pc.getProperties().setProperty(PigConfiguration.OPT_MULTIQUERY,""+true);
+        pc.getProperties().setProperty(PigConfiguration.PIG_OPT_MULTIQUERY,""+true);
 
         DataStorage dfs = pc.getDfs();
         dfs.setActiveContainer(dfs.asContainer("/tmp"));

Modified: pig/trunk/test/org/apache/pig/tez/TestSecondarySortTez.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/tez/TestSecondarySortTez.java?rev=1637758&r1=1637757&r2=1637758&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/tez/TestSecondarySortTez.java (original)
+++ pig/trunk/test/org/apache/pig/tez/TestSecondarySortTez.java Sun Nov  9 23:04:59 2014
@@ -47,12 +47,12 @@ public class TestSecondarySortTez extend
         TezCompiler comp = new TezCompiler(pp, pc);
         TezOperPlan tezPlan = comp.compile();
         boolean nocombiner = Boolean.parseBoolean(pc.getProperties().getProperty(
-                PigConfiguration.PROP_NO_COMBINER, "false"));
+                PigConfiguration.PIG_EXEC_NO_COMBINER, "false"));
 
         // Run CombinerOptimizer on Tez plan
         if (!nocombiner) {
             boolean doMapAgg = Boolean.parseBoolean(pc.getProperties()
-                    .getProperty(PigConfiguration.PROP_EXEC_MAP_PARTAGG,
+                    .getProperty(PigConfiguration.PIG_EXEC_MAP_PARTAGG,
                             "false"));
             CombinerOptimizer co = new CombinerOptimizer(tezPlan, doMapAgg);
             co.visit();