You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by xu...@apache.org on 2011/05/03 18:58:21 UTC

svn commit: r1099123 [1/16] - in /pig/branches/branch-0.9: ./ src/org/apache/pig/ src/org/apache/pig/backend/hadoop/executionengine/ src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/ src/org/apache/pig/impl/ src/org/apache/pig/impl/logi...

Author: xuefu
Date: Tue May  3 16:58:19 2011
New Revision: 1099123

URL: http://svn.apache.org/viewvc?rev=1099123&view=rev
Log:
PIG-1775: Removal of old logical plan

Modified:
    pig/branches/branch-0.9/CHANGES.txt
    pig/branches/branch-0.9/src/org/apache/pig/PigServer.java
    pig/branches/branch-0.9/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java
    pig/branches/branch-0.9/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java
    pig/branches/branch-0.9/src/org/apache/pig/impl/PigContext.java
    pig/branches/branch-0.9/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java
    pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LOCross.java
    pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LOLimit.java
    pig/branches/branch-0.9/src/org/apache/pig/parser/QueryLexer.g
    pig/branches/branch-0.9/test/org/apache/pig/parser/TestSchemaAliasVisitor.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestCollectedGroup.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestCombiner.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestFilterSimplification.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestGroupConstParallel.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestImplicitSplit.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestInputOutputFileValidator.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestJobSubmission.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestJoin.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestLoad.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestLogToPhyCompiler.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestLogicalOptimizer.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestLogicalPlanBuilder.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestMRCompiler.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestMapSideCogroup.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestMergeForEachOptimization.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestMergeJoin.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestMergeJoinOuter.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestNewPlanColumnPrune.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestNewPlanFilterAboveForeach.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestNewPlanFilterRule.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestNewPlanImplicitSplit.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestNewPlanLogToPhyTranslationVisitor.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestNewPlanPruneMapKeys.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestNewPlanPushDownForeachFlatten.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestOptimizeLimit.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestPartitionFilterOptimization.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestPartitionFilterPushDown.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestPigScriptParser.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestProjectionMap.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestPushDownForeachFlatten.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestPushUpFilter.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestRelevantFields.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestRequiredFields.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestRewire.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestSampleOptimizer.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestSchemaParser.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestSecondarySort.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestStore.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestTypeChecking.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestTypeCheckingValidator.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestTypeCheckingValidatorNewLP.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestTypeCheckingValidatorNoSchema.java
    pig/branches/branch-0.9/test/org/apache/pig/test/Util.java
    pig/branches/branch-0.9/test/org/apache/pig/test/data/DotFiles/new-optlimitplan10.dot
    pig/branches/branch-0.9/test/org/apache/pig/test/data/DotFiles/new-optlimitplan3.dot
    pig/branches/branch-0.9/test/org/apache/pig/test/data/DotFiles/new-optlimitplan4.dot
    pig/branches/branch-0.9/test/org/apache/pig/test/data/DotFiles/new-optlimitplan7.dot
    pig/branches/branch-0.9/test/org/apache/pig/test/data/GoldenFiles/Arithmetic.gld
    pig/branches/branch-0.9/test/org/apache/pig/test/data/GoldenFiles/BinCond.gld
    pig/branches/branch-0.9/test/org/apache/pig/test/data/GoldenFiles/Cogroup.gld
    pig/branches/branch-0.9/test/org/apache/pig/test/data/GoldenFiles/Comparison.gld
    pig/branches/branch-0.9/test/org/apache/pig/test/data/GoldenFiles/ComplexForeach.gld
    pig/branches/branch-0.9/test/org/apache/pig/test/data/GoldenFiles/Distinct.gld
    pig/branches/branch-0.9/test/org/apache/pig/test/data/GoldenFiles/Generate.gld
    pig/branches/branch-0.9/test/org/apache/pig/test/data/GoldenFiles/IsNull1.gld
    pig/branches/branch-0.9/test/org/apache/pig/test/data/GoldenFiles/Limit.gld
    pig/branches/branch-0.9/test/org/apache/pig/test/data/GoldenFiles/MRC18.gld
    pig/branches/branch-0.9/test/org/apache/pig/test/data/GoldenFiles/Sort.gld
    pig/branches/branch-0.9/test/org/apache/pig/test/data/GoldenFiles/Split1.gld
    pig/branches/branch-0.9/test/org/apache/pig/test/data/GoldenFiles/Union.gld
    pig/branches/branch-0.9/test/org/apache/pig/test/utils/LogicalPlanTester.java
    pig/branches/branch-0.9/test/org/apache/pig/test/utils/TypeCheckingTestUtil.java
    pig/branches/branch-0.9/test/org/apache/pig/test/utils/dotGraph/LogicalPlanLoader.java

Modified: pig/branches/branch-0.9/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/CHANGES.txt?rev=1099123&r1=1099122&r2=1099123&view=diff
==============================================================================
--- pig/branches/branch-0.9/CHANGES.txt (original)
+++ pig/branches/branch-0.9/CHANGES.txt Tue May  3 16:58:19 2011
@@ -343,6 +343,8 @@ PIG-1249: Safe-guards against misconfigu
 
 IMPROVEMENTS
 
+PIG-1775: Removal of old logical plan (xuefu)
+
 PIG-1998: Allow macro to return void (rding)
 
 PIG-1561: XMLLoader in Piggybank does not support bz2 or gzip compressed XML files (vivekp via daijy)

Modified: pig/branches/branch-0.9/src/org/apache/pig/PigServer.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/src/org/apache/pig/PigServer.java?rev=1099123&r1=1099122&r2=1099123&view=diff
==============================================================================
--- pig/branches/branch-0.9/src/org/apache/pig/PigServer.java (original)
+++ pig/branches/branch-0.9/src/org/apache/pig/PigServer.java Tue May  3 16:58:19 2011
@@ -44,8 +44,6 @@ import java.util.Set;
 import java.util.Stack;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import junit.framework.Assert;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.mapreduce.Job;
@@ -1406,9 +1404,9 @@ public class PigServer {
         private int processedStores = 0;
 
         private LogicalPlan lp;
-
-        private int currentLineNum = 0;
         
+        private int currentLineNum = 0;
+
         public Graph(boolean batchMode) {
             this.batchMode = batchMode;
             this.lp = new LogicalPlan();
@@ -1564,7 +1562,7 @@ public class PigServer {
         }
         
         /**
-         * Accumulate the given statement to previouis query statements and generate
+         * Accumulate the given statement to previous query statements and generate
          * an overall (raw) plan.
          */
         void registerQuery(String query, int startLine) throws IOException {

Modified: pig/branches/branch-0.9/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java?rev=1099123&r1=1099122&r2=1099123&view=diff
==============================================================================
--- pig/branches/branch-0.9/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java (original)
+++ pig/branches/branch-0.9/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java Tue May  3 16:58:19 2011
@@ -45,7 +45,6 @@ import org.apache.pig.backend.hadoop.exe
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.logicalLayer.FrontendException;
-import org.apache.pig.impl.logicalLayer.LogicalOperator;
 import org.apache.pig.impl.plan.OperatorKey;
 import org.apache.pig.impl.util.ObjectSerializer;
 import org.apache.pig.newplan.Operator;
@@ -86,7 +85,6 @@ public class HExecutionEngine {
     // map from LOGICAL key to into about the execution
     protected Map<OperatorKey, MapRedResult> materializedResults;
     
-    protected Map<LogicalOperator, LogicalRelationalOperator> opsMap;
     protected Map<Operator, PhysicalOperator> newLogToPhyMap;
     private LogicalPlan newPreoptimizedPlan;
     

Modified: pig/branches/branch-0.9/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java?rev=1099123&r1=1099122&r2=1099123&view=diff
==============================================================================
--- pig/branches/branch-0.9/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java (original)
+++ pig/branches/branch-0.9/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java Tue May  3 16:58:19 2011
@@ -57,13 +57,13 @@ import org.apache.pig.backend.hadoop.exe
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileSpec;
-import org.apache.pig.impl.logicalLayer.LogicalPlanBuilder;
 import org.apache.pig.impl.plan.CompilationMessageCollector;
 import org.apache.pig.impl.plan.PlanException;
 import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.plan.CompilationMessageCollector.MessageType;
 import org.apache.pig.impl.util.ConfigurationValidator;
 import org.apache.pig.impl.util.LogUtils;
+import org.apache.pig.parser.QueryParserDriver;
 import org.apache.pig.tools.pigstats.PigStats;
 import org.apache.pig.tools.pigstats.PigStatsUtil;
 import org.apache.pig.tools.pigstats.ScriptState;
@@ -222,7 +222,7 @@ public class MapReduceLauncher extends L
             Thread jcThread = new Thread(jc);
             jcThread.setUncaughtExceptionHandler(jctExceptionHandler);
             
-            jcThread.setContextClassLoader(LogicalPlanBuilder.classloader);
+            jcThread.setContextClassLoader(QueryParserDriver.classloader);
             
             //All the setup done, now lets launch the jobs.
             jcThread.start();

Modified: pig/branches/branch-0.9/src/org/apache/pig/impl/PigContext.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/src/org/apache/pig/impl/PigContext.java?rev=1099123&r1=1099122&r2=1099123&view=diff
==============================================================================
--- pig/branches/branch-0.9/src/org/apache/pig/impl/PigContext.java (original)
+++ pig/branches/branch-0.9/src/org/apache/pig/impl/PigContext.java Tue May  3 16:58:19 2011
@@ -50,10 +50,10 @@ import org.apache.pig.backend.executione
 import org.apache.pig.backend.hadoop.datastorage.HDataStorage;
 import org.apache.pig.backend.hadoop.executionengine.HExecutionEngine;
 import org.apache.pig.backend.hadoop.streaming.HadoopExecutableManager;
-import org.apache.pig.impl.logicalLayer.LogicalPlanBuilder;
 import org.apache.pig.impl.streaming.ExecutableManager;
 import org.apache.pig.impl.streaming.StreamingCommand;
 import org.apache.pig.impl.util.JarManager;
+import org.apache.pig.parser.QueryParserDriver;
 
 public class PigContext implements Serializable {
     private static final long serialVersionUID = 1L;
@@ -226,7 +226,7 @@ public class PigContext implements Seria
     public void addJar(URL resource) throws MalformedURLException{
         if (resource != null) {
             extraJars.add(resource);
-            LogicalPlanBuilder.classloader = createCl(null);
+            QueryParserDriver.classloader = createCl(null);
         }
     }
 
@@ -422,7 +422,7 @@ public class PigContext implements Seria
         for(String prefix: getPackageImportList()) {
             Class c;
             try {
-                c = Class.forName(prefix+name,true, LogicalPlanBuilder.classloader);
+                c = Class.forName(prefix+name,true, QueryParserDriver.classloader);
                 return c;
             } 
             catch (ClassNotFoundException e) {

Modified: pig/branches/branch-0.9/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java?rev=1099123&r1=1099122&r2=1099123&view=diff
==============================================================================
--- pig/branches/branch-0.9/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java (original)
+++ pig/branches/branch-0.9/src/org/apache/pig/impl/logicalLayer/LogicalPlanBuilder.java Tue May  3 16:58:19 2011
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.pig.impl.logicalLayer;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.pig.impl.PigContext;
-import org.apache.pig.impl.logicalLayer.parser.ParseException;
-import org.apache.pig.impl.logicalLayer.parser.QueryParser;
-import org.apache.pig.impl.plan.OperatorKey;
-
-
-/**
- * PlanBuilder class outputs a logical plan given a query String and set of ValidIDs
- * 
- */
-public class LogicalPlanBuilder {
-
-    public static ClassLoader classloader = LogicalPlanBuilder.class.getClassLoader();
-    private PigContext pigContext;
-    public LogicalPlanBuilder(PigContext pigContext) {
-        this.pigContext = pigContext;
-    }
-
-    public LogicalPlan parse(String scope, 
-                             String query, 
-                             Map<LogicalOperator, LogicalPlan> aliases,
-                             Map<OperatorKey, LogicalOperator> opTable,
-                             Map<String, LogicalOperator> aliasOp,
-                             Map<String, String> fileNameMap)
-        throws IOException, ParseException {
-        return parse(scope, query, aliases, opTable, aliasOp, 1, fileNameMap);
-   }
-    
-    public LogicalPlan parse(String scope, 
-                             String query, 
-                             Map<LogicalOperator, LogicalPlan> aliases,
-                             Map<OperatorKey, LogicalOperator> opTable,
-                             Map<String, LogicalOperator> aliasOp, 
-                             int start,
-                             Map<String, String> fileNameMap)
-        throws IOException, ParseException {
-        ByteArrayInputStream in = new ByteArrayInputStream(query.getBytes());        
-        QueryParser parser = new QueryParser(in, pigContext, scope, aliases, opTable, 
-                                             aliasOp, start, fileNameMap);
-        return parser.Parse();        
-    }
-
-}

Modified: pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LOCross.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LOCross.java?rev=1099123&r1=1099122&r2=1099123&view=diff
==============================================================================
--- pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LOCross.java (original)
+++ pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LOCross.java Tue May  3 16:58:19 2011
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.pig.impl.logicalLayer.FrontendException;
-import org.apache.pig.impl.logicalLayer.LogicalOperator;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.PlanVisitor;
 

Modified: pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LOLimit.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LOLimit.java?rev=1099123&r1=1099122&r2=1099123&view=diff
==============================================================================
--- pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LOLimit.java (original)
+++ pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LOLimit.java Tue May  3 16:58:19 2011
@@ -18,7 +18,6 @@
 package org.apache.pig.newplan.logical.relational;
 
 import org.apache.pig.impl.logicalLayer.FrontendException;
-import org.apache.pig.impl.logicalLayer.LogicalOperator;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.PlanVisitor;
 

Modified: pig/branches/branch-0.9/src/org/apache/pig/parser/QueryLexer.g
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/src/org/apache/pig/parser/QueryLexer.g?rev=1099123&r1=1099122&r2=1099123&view=diff
==============================================================================
--- pig/branches/branch-0.9/src/org/apache/pig/parser/QueryLexer.g (original)
+++ pig/branches/branch-0.9/src/org/apache/pig/parser/QueryLexer.g Tue May  3 16:58:19 2011
@@ -308,7 +308,7 @@ FLOATNUMBER : DOUBLENUMBER ( 'F' )?
 
 QUOTEDSTRING :  '\'' (   ( ~ ( '\'' | '\\' | '\n' | '\r' ) )
                        | ( '\\' ( ( 'N' | 'T' | 'B' | 'R' | 'F' | '\\' | '\'' ) ) )
-                       | ( '\\u' ( '0'..'9' | 'A'..'F' )
+                       | ( '\\U' ( '0'..'9' | 'A'..'F' )
                                  ( '0'..'9' | 'A'..'F' )
                                  ( '0'..'9' | 'A'..'F' )
                                  ( '0'..'9' | 'A'..'F' )  )
@@ -318,7 +318,7 @@ QUOTEDSTRING :  '\'' (   ( ~ ( '\'' | '\
 
 MULTILINE_QUOTEDSTRING :  '\'' (   ( ~ ( '\'' | '\\' ) )
                                  | ( '\\' ( ( 'N' | 'T' | 'B' | 'R' | 'F' | '\\' | '\'' | 'n' | 'r' ) ) )
-                                 | ( '\\u' ( '0'..'9' | 'A'..'F' )
+                                 | ( '\\U' ( '0'..'9' | 'A'..'F' )
                                            ( '0'..'9' | 'A'..'F' )
                                            ( '0'..'9' | 'A'..'F' )
                                            ( '0'..'9' | 'A'..'F' )  )

Modified: pig/branches/branch-0.9/test/org/apache/pig/parser/TestSchemaAliasVisitor.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/test/org/apache/pig/parser/TestSchemaAliasVisitor.java?rev=1099123&r1=1099122&r2=1099123&view=diff
==============================================================================
--- pig/branches/branch-0.9/test/org/apache/pig/parser/TestSchemaAliasVisitor.java (original)
+++ pig/branches/branch-0.9/test/org/apache/pig/parser/TestSchemaAliasVisitor.java Tue May  3 16:58:19 2011
@@ -39,7 +39,7 @@ public class TestSchemaAliasVisitor {
         validate( query );
     }
     
-   @Test
+    @Test
     public void testNegative1() throws RecognitionException, ParsingFailureException, IOException {
         String query = "A = load 'x' as ( u:int, v:long, w:bytearray); " + 
                        "B = foreach A generate $0, v, $0; " +
@@ -47,12 +47,40 @@ public class TestSchemaAliasVisitor {
         try {
             validate( query );
         } catch(PlanValidationException ex) {
-        Assert.assertTrue( ex.getMessage().contains( "Duplicate schema alias" ) );
+            Assert.assertTrue( ex.getMessage().contains( "Duplicate schema alias" ) );
+            return;
+        }
+        Assert.fail( "Query should fail to validate." );
+    }
+
+    @Test
+    // See PIG-644
+    public void testNegative2() throws RecognitionException, ParsingFailureException, IOException {
+         String query = "a = load '1.txt' as (a0:int, a1:int);" + 
+                        "b = foreach a generate a0, a1 as a0;" +
+                        "c = store b into 'output';";
+         try {
+             validate( query );
+         } catch(PlanValidationException ex) {
+             Assert.assertTrue( ex.getMessage().contains( "Duplicate schema alias" ) );
+             return;
+         }
+         Assert.fail( "Query should fail to validate." );
+     }
+     
+    @Test
+    // See PIG-644
+    public void testNegative3() throws RecognitionException, ParsingFailureException, IOException {
+        String query = "a = load '1.txt' as (a0:int, a0:int);" + 
+                       "store a into 'output';";
+        try {
+            validate( query );
+        } catch(RecognitionException ex) {
+            Assert.assertTrue( ex.toString().contains( "Duplicated alias in schema" ) );
             return;
         }
         Assert.fail( "Query should fail to validate." );
     }
-    
 
     private LogicalPlan validate(String query) throws RecognitionException, ParsingFailureException, IOException {
         LogicalPlan plan = ParserTestingUtils.generateLogicalPlan( query );

Modified: pig/branches/branch-0.9/test/org/apache/pig/test/TestCollectedGroup.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/test/org/apache/pig/test/TestCollectedGroup.java?rev=1099123&r1=1099122&r2=1099123&view=diff
==============================================================================
--- pig/branches/branch-0.9/test/org/apache/pig/test/TestCollectedGroup.java (original)
+++ pig/branches/branch-0.9/test/org/apache/pig/test/TestCollectedGroup.java Tue May  3 16:58:19 2011
@@ -30,16 +30,16 @@ import org.apache.pig.backend.executione
 import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.Tuple;
-import org.apache.pig.test.utils.LogicalPlanTester;
 import org.apache.pig.test.utils.TestHelper;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompilerException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POCollectedGroup;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.impl.PigContext;
-import org.apache.pig.impl.logicalLayer.LOCogroup;
-import org.apache.pig.impl.logicalLayer.LogicalPlan;
 import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.newplan.logical.relational.LOCogroup;
+import org.apache.pig.newplan.logical.relational.LOStore;
+import org.apache.pig.newplan.logical.relational.LogicalPlan;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -87,13 +87,12 @@ public class TestCollectedGroup {
     
     @Test
     public void testNonCollectableLoader() throws Exception{
-        LogicalPlanTester lpt = new LogicalPlanTester();
-        lpt.buildPlan("A = LOAD '" + INPUT_FILE + "' as (id, name, grade);");
-        LogicalPlan lp = lpt.buildPlan("B = group A by id using 'collected';");
+        String query = "A = LOAD '" + INPUT_FILE + "' as (id, name, grade);" +
+                       "B = group A by id using 'collected';";
         PigContext pc = new PigContext(ExecType.MAPREDUCE,cluster.getProperties());
         pc.connect();
         try {
-            Util.buildMRPlan(Util.buildPhysicalPlan(lp, pc),pc);  
+            Util.buildMRPlan(Util.buildPp(pigServer, query),pc);  
             Assert.fail("Must throw MRCompiler Exception");
         } catch (Exception e) {
             Assert.assertTrue(e instanceof MRCompilerException);
@@ -101,21 +100,25 @@ public class TestCollectedGroup {
     }
 
     @Test
-    public void testCollectedGrpSpecifiedInSingleQuotes1(){
-        
-        LogicalPlanTester lpt = new LogicalPlanTester();
-        lpt.buildPlan("A = LOAD '" + INPUT_FILE + "' as (id, name, grade);");
-        LogicalPlan lp = lpt.buildPlan("B = group A by id using 'collected';");
-        Assert.assertEquals(LOCogroup.GROUPTYPE.COLLECTED, ((LOCogroup)lp.getLeaves().get(0)).getGroupType());
+    public void testCollectedGrpSpecifiedInSingleQuotes1() throws Exception {
+    	String query = "A = LOAD '" + INPUT_FILE + "' as (id, name, grade);" + 
+    	               "B = group A by id using 'collected';" +
+    	               "Store B into 'y';";
+        LogicalPlan lp = Util.buildLp(pigServer, query );
+        LOStore store = (LOStore)lp.getSinks().get(0);
+        LOCogroup grp = (LOCogroup)lp.getPredecessors( store ).get(0);
+        Assert.assertEquals( LOCogroup.GROUPTYPE.COLLECTED, grp.getGroupType() );
     }
     
     @Test
-    public void testCollectedGrpSpecifiedInSingleQuotes2(){
-        
-        LogicalPlanTester lpt = new LogicalPlanTester();
-        lpt.buildPlan("A = LOAD '" + INPUT_FILE + "' as (id, name, grade);");
-        LogicalPlan lp = lpt.buildPlan("B = group A all using 'regular';");
-        Assert.assertEquals(LOCogroup.GROUPTYPE.REGULAR, ((LOCogroup)lp.getLeaves().get(0)).getGroupType());
+    public void testCollectedGrpSpecifiedInSingleQuotes2() throws Exception{
+        String query = "A = LOAD '" + INPUT_FILE + "' as (id, name, grade);" +
+                       "B = group A all using 'regular';" +
+                       "Store B into 'y';";
+        LogicalPlan lp = Util.buildLp(pigServer, query );
+        LOStore store = (LOStore)lp.getSinks().get(0);
+        LOCogroup grp = (LOCogroup)lp.getPredecessors( store ).get(0);
+        Assert.assertEquals(LOCogroup.GROUPTYPE.REGULAR, grp.getGroupType());
     }
     
     @AfterClass

Modified: pig/branches/branch-0.9/test/org/apache/pig/test/TestCombiner.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/test/org/apache/pig/test/TestCombiner.java?rev=1099123&r1=1099122&r2=1099123&view=diff
==============================================================================
--- pig/branches/branch-0.9/test/org/apache/pig/test/TestCombiner.java (original)
+++ pig/branches/branch-0.9/test/org/apache/pig/test/TestCombiner.java Tue May  3 16:58:19 2011
@@ -38,11 +38,11 @@ import org.apache.pig.EvalFunc;
 import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.PigStorage;
+import org.apache.pig.data.DataBag;
+import org.apache.pig.data.DefaultDataBag;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileLocalizer;
-import org.apache.pig.impl.logicalLayer.LogicalPlan;
-import org.apache.pig.test.utils.LogicalPlanTester;
 import org.junit.AfterClass;
 import org.junit.Test;
 
@@ -58,28 +58,26 @@ public class TestCombiner {
     
     @Test
     public void testSuccessiveUserFuncs1() throws Exception{
-        
-        LogicalPlanTester tester = new LogicalPlanTester();
-        tester.buildPlan( "a = load 'students.txt' as (c1,c2,c3,c4); ");
-        tester.buildPlan("c = group a by c2; ");
-        tester.buildPlan("f = foreach c generate COUNT(org.apache.pig.builtin.Distinct($1.$2)); ");
-        LogicalPlan lp = tester.buildPlan("store f into 'out';");
-        PigContext pc = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()).getPigContext();
-        assertTrue((Util.buildMRPlan(Util.buildPhysicalPlan(lp,pc),pc).getRoots().get(0).combinePlan.isEmpty()));
+        String query = "a = load 'students.txt' as (c1,c2,c3,c4); " +
+                       "c = group a by c2; " +
+                       "f = foreach c generate COUNT(org.apache.pig.builtin.Distinct($1.$2)); " +
+                       "store f into 'out';";
+        PigServer pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigContext pc = pigServer.getPigContext();
+        assertTrue((Util.buildMRPlan(Util.buildPp(pigServer,query),pc).getRoots().get(0).combinePlan.isEmpty()));
     }
 
     @Test
-    public void testSuccessiveUserFuncs2() throws Exception{
-        
-        LogicalPlanTester tester = new LogicalPlanTester();
-        tester.buildPlan( "a = load 'students.txt' as (c1,c2,c3,c4); ");
-        tester.buildPlan("c = group a by c2; ");
+    public void testSuccessiveUserFuncs2() throws Exception {
         String dummyUDF = JiraPig1030.class.getName();
-        tester.buildPlan("f = foreach c generate COUNT("+dummyUDF+"" +
-        		"(org.apache.pig.builtin.Distinct($1.$2),"+dummyUDF+"())); ");
-        LogicalPlan lp = tester.buildPlan("store f into 'out';");
-        PigContext pc = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()).getPigContext();
-        assertTrue((Util.buildMRPlan(Util.buildPhysicalPlan(lp,pc),pc).getRoots().get(0).combinePlan.isEmpty()));
+    	String query = "a = load 'students.txt' as (c1,c2,c3,c4); " +
+                       "c = group a by c2; " +
+                       "f = foreach c generate COUNT(" + dummyUDF + "" +
+               		   "(org.apache.pig.builtin.Distinct($1.$2),"+dummyUDF+"())); " +
+               		   "store f into 'out';";
+        PigServer pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigContext pc = pigServer.getPigContext();
+        assertTrue((Util.buildMRPlan(Util.buildPp(pigServer,query),pc).getRoots().get(0).combinePlan.isEmpty()));
     }
     
     @Test
@@ -516,10 +514,10 @@ public class TestCombiner {
         }
     }
 
-    public static class JiraPig1030 extends EvalFunc<String> {
+    public static class JiraPig1030 extends EvalFunc<DataBag> {
         
-        public String exec(Tuple input) throws IOException {
-            return "";
+        public DataBag exec(Tuple input) throws IOException {
+            return new DefaultDataBag();
         }
     }
    

Modified: pig/branches/branch-0.9/test/org/apache/pig/test/TestFilterSimplification.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/test/org/apache/pig/test/TestFilterSimplification.java?rev=1099123&r1=1099122&r2=1099123&view=diff
==============================================================================
--- pig/branches/branch-0.9/test/org/apache/pig/test/TestFilterSimplification.java (original)
+++ pig/branches/branch-0.9/test/org/apache/pig/test/TestFilterSimplification.java Tue May  3 16:58:19 2011
@@ -21,7 +21,7 @@ package org.apache.pig.test;
 import java.util.*;
 
 import org.apache.pig.ExecType;
-import org.apache.pig.newplan.logical.LogicalPlanMigrationVistor;
+import org.apache.pig.PigServer;
 import org.apache.pig.newplan.logical.optimizer.LogicalPlanOptimizer;
 import org.apache.pig.newplan.logical.relational.LogicalPlan;
 import org.apache.pig.newplan.logical.rules.LogicalExpressionSimplifier;
@@ -29,521 +29,454 @@ import org.apache.pig.newplan.OperatorPl
 import org.apache.pig.newplan.optimizer.PlanOptimizer;
 import org.apache.pig.newplan.optimizer.Rule;
 import org.apache.pig.impl.PigContext;
-import org.apache.pig.impl.plan.VisitorException;
-import org.apache.pig.test.utils.LogicalPlanTester;
 
 import junit.framework.TestCase;
+
+import org.junit.Before;
 import org.junit.Test;
 
 public class TestFilterSimplification extends TestCase {
 
     LogicalPlan plan = null;
-    PigContext pc = new PigContext(ExecType.LOCAL, new Properties());
+    PigContext pc = null;
+    PigServer pigServer = null;
 
-    private LogicalPlan migratePlan(
-                    org.apache.pig.impl.logicalLayer.LogicalPlan lp)
-                    throws VisitorException {
-        LogicalPlanMigrationVistor visitor = new LogicalPlanMigrationVistor(lp);
-        visitor.visit();
-        org.apache.pig.newplan.logical.relational.LogicalPlan newplan = visitor.getNewLogicalPlan();
-        return newplan;
+    @Before
+    public void setUp() throws Exception {
+        pigServer = new PigServer(ExecType.LOCAL, new Properties());
+        pc = pigServer.getPigContext();
     }
-
+    
     @Test
     public void test1() throws Exception {
         // case 1: simple and implication
-        LogicalPlanTester lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) AND (id > 5);");
-        org.apache.pig.impl.logicalLayer.LogicalPlan plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan newLogicalPlan = migratePlan(plan);
+        String query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) AND (id > 5);" + 
+                       "store b into 'empty';";
+        LogicalPlan newLogicalPlan = Util.buildLp(pigServer, query);
 
         PlanOptimizer optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 5);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 5);" +
+                "store b into 'empty';";
+        LogicalPlan expected = Util.buildLp(pigServer, query);
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 2: simple or implication
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) OR (id > 5);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) OR (id > 5);" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 3: constant expression eval
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3+4*2);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3+4*2);" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 11);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 11);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 4: simple NOT 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by NOT(NOT(NOT(id > 3)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by NOT(NOT(NOT(id > 3)));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id <= 3);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id <= 3);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 5: redundant NOT 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by NOT(NOT(id > 3));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by NOT(NOT(id > 3));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 6: negative
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) AND (v1 is null);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) AND (v1 is null);" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) AND (v1 is null);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) AND (v1 is null);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 7: is not null
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by NOT(v1 is null);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by NOT(v1 is null);" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (v1 is not null);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (v1 is not null);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 8: combo I
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by NOT((id > 1) OR ((v1 is null) AND (id > 5)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by NOT((id > 1) OR ((v1 is null) AND (id > 5)));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id <= 1);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id <= 1);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 9: combo II: lhs <-> rhs
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by NOT(((id > 5) AND (v1 is null)) OR (id > 1));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by NOT(((id > 5) AND (v1 is null)) OR (id > 1));"+
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id <= 1);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id <= 1);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 10: complementary OR
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) OR (id >= 1));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) OR (id >= 1));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = load 'd.txt' as (id:int, v1, v2);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = load 'd.txt' as (id:int, v1, v2);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 11: OR Equality elimination 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) OR (id < 1));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) OR (id < 1));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id < 1);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id < 1);" +
+        "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 12: AND Equality elimination 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) AND (id < 1));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) AND (id < 1));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id < 1);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id < 1);" + "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 13: negative case 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) AND (v1 is null));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) AND (v1 is null));" + "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) AND (v1 is NULL));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) AND (v1 is NULL));" + "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 14: combo III
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by NOT((id > 1) OR ((v1 is null) AND (id > 1+2*2)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by NOT((id > 1) OR ((v1 is null) AND (id > 1+2*2)));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id <= 1);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id <= 1);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 15: combo III: negative
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 5) OR (v1 < 3)) AND ((id > 4) OR (v1 > 5)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 5) OR (v1 < 3)) AND ((id > 4) OR (v1 > 5)));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 5) OR (v1 < 3)) AND ((id > 4) OR (v1 > 5)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 5) OR (v1 < 3)) AND ((id > 4) OR (v1 > 5)));" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 15: combo III: negative
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 5) OR (v1 > 3)) AND ((id > 4) OR (v1 > 5)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 5) OR (v1 > 3)) AND ((id > 4) OR (v1 > 5)));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 5) OR (v1 > 3)) AND ((id > 4) OR (v1 > 5)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 5) OR (v1 > 3)) AND ((id > 4) OR (v1 > 5)));" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 16: conflicting OR
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) OR (id > 1));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) OR (id > 1));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) OR (id > 1));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) OR (id > 1));" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 17: conflicting AND: negtive case for now
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) AND (id > 1));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) AND (id > 1));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) AND (id > 1));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by ((id < 1) AND (id > 1));" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 18: combo IV: negative
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 5) OR (v1 > 3)) AND ((id < 8) OR (v1 > 5)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 5) OR (v1 > 3)) AND ((id < 8) OR (v1 > 5)));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 5) OR (v1 > 3)) AND ((id < 8) OR (v1 > 5)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 5) OR (v1 > 3)) AND ((id < 8) OR (v1 > 5)));" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 19: negative AND
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) AND (id < 5);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) AND (id < 5);" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) AND (id < 5);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) AND (id < 5);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 20: negative OR
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) OR (id < 5);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) OR (id < 5);" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) OR (id < 5);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (id > 3) OR (id < 5);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 20: combo V: negative
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((v1 > 3) OR (id > 5)) AND ((id < 8) OR (v1 > 5)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((v1 > 3) OR (id > 5)) AND ((id < 8) OR (v1 > 5)));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((v1 > 3) OR (id > 5)) AND ((id < 8) OR (v1 > 5)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((v1 > 3) OR (id > 5)) AND ((id < 8) OR (v1 > 5)));" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 22: combo V: negative
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((v1 > 3) OR (id > 5)) AND ((v1 > 5) OR (id < 8)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((v1 > 3) OR (id > 5)) AND ((v1 > 5) OR (id < 8)));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((v1 > 3) OR (id > 5)) AND ((v1 > 5) OR (id < 8)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((v1 > 3) OR (id > 5)) AND ((v1 > 5) OR (id < 8)));" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 23: combo VI: extremely degenerate 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by ((((id > 1) OR (id > 2)) AND ((id > 3) OR (id > 4))) AND (((id > 5) OR (id > 6)) AND ((id > 7) OR (id > 8))));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by ((((id > 1) OR (id > 2)) AND ((id > 3) OR (id > 4))) AND (((id > 5) OR (id > 6)) AND ((id > 7) OR (id > 8))));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (id > 7);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (id > 7);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 24: combo VII: extremely degenerate 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by ((((id > 1) OR (id > 2)) AND ((id > 3) OR (id > 4))) AND (id > 7));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by ((((id > 1) OR (id > 2)) AND ((id > 3) OR (id > 4))) AND (id > 7));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (id > 7);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (id > 7);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 25: combo VII: extremely degenerate 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by ((((id > 1) OR (id > 2)) AND ((id > 3) OR (id > 4))) AND (((id > 5) AND (id > 7))));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by ((((id > 1) OR (id > 2)) AND ((id > 3) OR (id > 4))) AND (((id > 5) AND (id > 7))));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (id > 7);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (id > 7);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 26: combo VIII: lhs<->rhs for case 25 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by ((((id > 7) AND (id > 5))) AND (((id > 4) OR (id > 3)) AND ((id > 2) OR (id > 1))));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by ((((id > 7) AND (id > 5))) AND (((id > 4) OR (id > 3)) AND ((id > 2) OR (id > 1))));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (id > 7);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (id > 7);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 27: combo VII: rhs<->lhs for case 24 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by ((id > 7) AND (((id > 4) OR (id > 3)) AND ((id > 2) OR (id > 1))));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by ((id > 7) AND (((id > 4) OR (id > 3)) AND ((id > 2) OR (id > 1))));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (id > 7);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (id > 7);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 28: complex equality 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 4) OR (id > 3)) AND ((id > 3) OR (id > 4)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 4) OR (id > 3)) AND ((id > 3) OR (id > 4)));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (id > 3);");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (id > 3);" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 29: complex equality 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 4) OR (v1 > 3)) AND ((v1 > 3) OR (id > 4)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 4) OR (v1 > 3)) AND ((v1 > 3) OR (id > 4)));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by ((id > 4) OR (v1 > 3));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by ((id > 4) OR (v1 > 3));" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // case 30: complex equality 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 4) OR (v1 > 3)) OR ((v1 > 3) OR (id > 4)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by (((id > 4) OR (v1 > 3)) OR ((v1 > 3) OR (id > 4)));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1:int, v2)) by ((id > 4) OR (v1 > 3));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1:int, v2)) by ((id > 4) OR (v1 > 3));" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
@@ -551,66 +484,58 @@ public class TestFilterSimplification ex
 
     @Test
     public void test2() throws Exception {
-        LogicalPlanTester lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (name, age, gpa)) by age >= 50 or name > 'fred' and gpa <= 3.0 or name >= 'bob';");
-        org.apache.pig.impl.logicalLayer.LogicalPlan plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan newLogicalPlan = migratePlan(plan);
+        String query = "b = filter (load 'd.txt' as (name, age, gpa)) by age >= 50 or name > 'fred' and gpa <= 3.0 or name >= 'bob';" +
+                       "store b into 'empty';";
+        LogicalPlan newLogicalPlan = Util.buildLp(pigServer, query);;
 
         PlanOptimizer optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (name, age, gpa)) by age >= 50 or name >= 'bob';");
-        plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (name, age, gpa)) by age >= 50 or name >= 'bob';" +
+                "store b into 'empty';";
+        LogicalPlan expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
         
         // Regex filtering
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (name:chararray, age:int, registration, contributions:double)) by (name matches '^fred.*' and (chararray)registration matches '^dem.*');");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (name:chararray, age:int, registration, contributions:double)) by (name matches '^fred.*' and (chararray)registration matches '^dem.*');" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (name:chararray, age:int, registration, contributions:double)) by (name matches '^fred.*' and (chararray)registration matches '^dem.*');");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (name:chararray, age:int, registration, contributions:double)) by (name matches '^fred.*' and (chararray)registration matches '^dem.*');" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
         
         // NOT Regex filtering 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt') by (not $0 matches '^fred.*');");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt') by (not $0 matches '^fred.*');" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt') by (not $0 matches '^fred.*');");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt') by (not $0 matches '^fred.*');" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));   
         
         // naiive filtering
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt') by 1==1;");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt') by 1==1;" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = load 'd.txt';");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = load 'd.txt';" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
     }
@@ -618,85 +543,75 @@ public class TestFilterSimplification ex
     @Test
     public void test3() throws Exception {
         // boolean constant elimination: AND
-        LogicalPlanTester lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by ((v1 is not null) AND (id == 1) AND (1 == 1));");
-        org.apache.pig.impl.logicalLayer.LogicalPlan plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan newLogicalPlan = migratePlan(plan);
+        String query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by ((v1 is not null) AND (id == 1) AND (1 == 1));" +
+                       "store b into 'empty';";
+        LogicalPlan newLogicalPlan = Util.buildLp(pigServer, query);;
 
         PlanOptimizer optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by ((v1 is not null) AND (id == 1));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by ((v1 is not null) AND (id == 1));" +
+                "store b into 'empty';";
+        LogicalPlan expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // boolean constant elimination: OR
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by (((v1 is not null) AND (id == 1)) OR (1 == 0));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by (((v1 is not null) AND (id == 1)) OR (1 == 0));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by ((v1 is not null) AND (id == 1));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by ((v1 is not null) AND (id == 1));" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
         
         // the mirror case of the above
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by ((1 == 0) OR ((v1 is not null) AND (id == 1)));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by ((1 == 0) OR ((v1 is not null) AND (id == 1)));" +
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (id:int, v1, v2)) by ((v1 is not null) AND (id == 1));");
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (id:int, v1, v2)) by ((v1 is not null) AND (id == 1));" +
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
     }
 
     @Test
     public void test4() throws Exception {
-        LogicalPlanTester lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (a:chararray, b:long, c:map[], d:chararray, e:chararray)) by a == 'v' and b == 117L and c#'p1' == 'h' and c#'p2' == 'to' and ((d is not null and d != '') or (e is not null and e != ''));"); 
-
-        org.apache.pig.impl.logicalLayer.LogicalPlan plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan newLogicalPlan = migratePlan(plan);
+        String query = "b = filter (load 'd.txt' as (a:chararray, b:long, c:map[], d:chararray, e:chararray)) by a == 'v' and b == 117L and c#'p1' == 'h' and c#'p2' == 'to' and ((d is not null and d != '') or (e is not null and e != ''));" + 
+                       "store b into 'empty';";
+        LogicalPlan newLogicalPlan = Util.buildLp(pigServer, query);;
 
         PlanOptimizer optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (a:chararray, b:long, c:map[], d:chararray, e:chararray)) by a == 'v' and b == 117L and c#'p1' == 'h' and c#'p2' == 'to' and ((d is not null and d != '') or (e is not null and e != ''));"); 
-        plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (a:chararray, b:long, c:map[], d:chararray, e:chararray)) by a == 'v' and b == 117L and c#'p1' == 'h' and c#'p2' == 'to' and ((d is not null and d != '') or (e is not null and e != ''));" + 
+                "store b into 'empty';";
+        LogicalPlan expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
         // mirror of the above
-        lpt.buildPlan("b = filter (load 'd.txt' as (a:chararray, b:long, c:map[], d:chararray, e:chararray)) by ((d is not null and d != '') or (e is not null and e != '')) and a == 'v' and b == 117L and c#'p1' == 'h' and c#'p2' == 'to';"); 
+        query = "b = filter (load 'd.txt' as (a:chararray, b:long, c:map[], d:chararray, e:chararray)) by ((d is not null and d != '') or (e is not null and e != '')) and a == 'v' and b == 117L and c#'p1' == 'h' and c#'p2' == 'to';" + 
 
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (a:chararray, b:long, c:map[], d:chararray, e:chararray)) by ((d is not null and d != '') or (e is not null and e != '')) and a == 'v' and b == 117L and c#'p1' == 'h' and c#'p2' == 'to';"); 
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (a:chararray, b:long, c:map[], d:chararray, e:chararray)) by ((d is not null and d != '') or (e is not null and e != '')) and a == 'v' and b == 117L and c#'p1' == 'h' and c#'p2' == 'to';" + 
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
 
@@ -715,21 +630,13 @@ public class TestFilterSimplification ex
     private void comboRunner2(boolean b1, boolean b2, boolean b3) throws Exception {
         StringBuilder sb = new StringBuilder();
         sb.append("b = filter (load 'd.txt' as (a:int, b:int, c:int, d:int)) by (((a < 1) " + (b1 ? "and" : "or") + " (b < 2)) " + (b2 ? "and" : "or") + " ((c < 3) " + (b3 ? "and" : "or") + " (d < 4)));");  
-        String query = sb.toString();
-
-        LogicalPlanTester lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan(query); 
-
-        org.apache.pig.impl.logicalLayer.LogicalPlan plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan newLogicalPlan = migratePlan(plan);
+        String query = sb.toString() + "store b into 'empty';";
+        LogicalPlan newLogicalPlan = Util.buildLp(pigServer, query);;
 
         PlanOptimizer optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan(query); 
-        plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan expected = migratePlan(plan);
+        LogicalPlan expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
     }
@@ -751,55 +658,43 @@ public class TestFilterSimplification ex
     private void comboRunner3(boolean b1, boolean b2, boolean b3, boolean b4, boolean b5, boolean b6, boolean b7) throws Exception {
         StringBuilder sb = new StringBuilder();
         sb.append("b = filter (load 'd.txt' as (a:int, b:int, c:int, d:int, e:int, f:int, g:int, h:int)) by ((((a < 1) " + (b1 ? "and" : "or") + " (b < 2)) " + (b2 ? "and" : "or") + " ((c < 3) " + (b3 ? "and" : "or") + " (d < 4))) " + (b4 ? "and" : "or") + " (((e < 5) " + (b5 ? "and" : "or") + " (f < 6)) " + (b6 ? "and" : "or") + " ((g < 7) " + (b7 ? "and" : "or") + " (h < 8))));");  
-        String query = sb.toString();
-
-        LogicalPlanTester lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan(query); 
-
-        org.apache.pig.impl.logicalLayer.LogicalPlan plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan newLogicalPlan = migratePlan(plan);
+        String query = sb.toString() + "store b into 'empty';";
+        LogicalPlan newLogicalPlan = Util.buildLp(pigServer, query);;
 
         PlanOptimizer optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan(query); 
-        plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan expected = migratePlan(plan);
+        LogicalPlan expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
     }
 
     @Test
     public void test7() throws Exception {
-        LogicalPlanTester lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (k1, k2, k3, v1, v2, v3)) by k2#'f1'#'f' is not null and (v2#'f'#'f1' is not null or v2#'f'#'f2' is not null);"); 
-
-        org.apache.pig.impl.logicalLayer.LogicalPlan plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan newLogicalPlan = migratePlan(plan);
+        String query = "b = filter (load 'd.txt' as (k1, k2, k3, v1, v2, v3)) by k2#'f1'#'f' is not null and (v2#'f'#'f1' is not null or v2#'f'#'f2' is not null);" + 
+                       "store b into 'empty';";
+        LogicalPlan newLogicalPlan = Util.buildLp(pigServer, query);;
 
         PlanOptimizer optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (k1, k2, k3, v1, v2, v3)) by k2#'f1'#'f' is not null and (v2#'f'#'f1' is not null or v2#'f'#'f2' is not null);"); 
-        plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (k1, k2, k3, v1, v2, v3)) by k2#'f1'#'f' is not null and (v2#'f'#'f1' is not null or v2#'f'#'f2' is not null);" + 
+                "store b into 'empty';";
+        LogicalPlan expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
         
-        lpt.buildPlan("b = filter (load 'd.txt' as (k1, k2, k3, v1, v2, v3)) by k2#'f1'#'f' is not null and (v2#'f1'#'f' is not null or v2#'f2'#'f' is not null);"); 
+        query = "b = filter (load 'd.txt' as (k1, k2, k3, v1, v2, v3)) by k2#'f1'#'f' is not null and (v2#'f1'#'f' is not null or v2#'f2'#'f' is not null);" + 
 
-        plan = lpt.buildPlan("store b into 'empty';");
-        newLogicalPlan = migratePlan(plan);
+                "store b into 'empty';";
+        newLogicalPlan = Util.buildLp(pigServer, query);;
 
         optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (k1, k2, k3, v1, v2, v3)) by k2#'f1'#'f' is not null and (v2#'f1'#'f' is not null or v2#'f2'#'f' is not null);"); 
-        plan = lpt.buildPlan("store b into 'empty';");
-        expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (k1, k2, k3, v1, v2, v3)) by k2#'f1'#'f' is not null and (v2#'f1'#'f' is not null or v2#'f2'#'f' is not null);" + 
+                "store b into 'empty';";
+        expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
     }
@@ -807,19 +702,16 @@ public class TestFilterSimplification ex
     @Test
     // PIG-1820
     public void test8() throws Exception {
-        LogicalPlanTester lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (a0, a1)) by (a0 is not null or a1 is not null) and IsEmpty(a0);"); 
-
-        org.apache.pig.impl.logicalLayer.LogicalPlan plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan newLogicalPlan = migratePlan(plan);
+        String query = "b = filter (load 'd.txt' as (a0, a1)) by (a0 is not null or a1 is not null) and IsEmpty(a0);" + 
+                       "store b into 'empty';";
+        LogicalPlan newLogicalPlan = Util.buildLp(pigServer, query);;
 
         PlanOptimizer optimizer = new MyPlanOptimizer(newLogicalPlan, 10);
         optimizer.optimize();
 
-        lpt = new LogicalPlanTester(pc);
-        lpt.buildPlan("b = filter (load 'd.txt' as (a0, a1)) by (a0 is not null or a1 is not null) and IsEmpty(a0);"); 
-        plan = lpt.buildPlan("store b into 'empty';");
-        LogicalPlan expected = migratePlan(plan);
+        query = "b = filter (load 'd.txt' as (a0, a1)) by (a0 is not null or a1 is not null) and IsEmpty(a0);" + 
+                "store b into 'empty';";
+        LogicalPlan expected = Util.buildLp(pigServer, query);;
 
         assertTrue(expected.isEqual(newLogicalPlan));
     }

Modified: pig/branches/branch-0.9/test/org/apache/pig/test/TestGroupConstParallel.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/test/org/apache/pig/test/TestGroupConstParallel.java?rev=1099123&r1=1099122&r2=1099123&view=diff
==============================================================================
--- pig/branches/branch-0.9/test/org/apache/pig/test/TestGroupConstParallel.java (original)
+++ pig/branches/branch-0.9/test/org/apache/pig/test/TestGroupConstParallel.java Tue May  3 16:58:19 2011
@@ -37,14 +37,12 @@ import org.apache.pig.backend.hadoop.exe
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.PigContext;
-import org.apache.pig.impl.logicalLayer.LogicalPlan;
 import org.apache.pig.impl.util.ConfigurationValidator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.logical.optimizer.LogicalPlanOptimizer;
 import org.apache.pig.newplan.logical.rules.GroupByConstParallelSetter;
 import org.apache.pig.newplan.optimizer.Rule;
 import org.apache.pig.test.utils.GenPhyOp;
-import org.apache.pig.test.utils.LogicalPlanTester;
 import org.apache.pig.tools.pigstats.JobStats;
 import org.apache.pig.tools.pigstats.PigStats;
 import org.apache.pig.tools.pigstats.PigStats.JobGraph;
@@ -116,13 +114,10 @@ public class TestGroupConstParallel {
         pc.defaultParallel = 100;
         pc.connect();
         
-        LogicalPlanTester planTester = new LogicalPlanTester() ;
-        planTester.buildPlan("a = load 'input';");
-        LogicalPlan lp = planTester.buildPlan("b = group a by 1;");
-        
-        PhysicalPlan pp = Util.getNewOptimizedPhysicalPlan(lp, pc);
-        POStore store = GenPhyOp.dummyPigStorageOp();
-        pp.addAsLeaf(store);
+        String query = "a = load 'input';\n" + "b = group a by 1;" + "store b into 'output';";
+        PigServer pigServer = new PigServer( ExecType.MAPREDUCE, cluster.getProperties() );
+        PhysicalPlan pp = Util.buildPp( pigServer, query );
+        
         MROperPlan mrPlan = Util.buildMRPlan(pp, pc);
 
         ConfigurationValidator.validatePigProperties(pc.getProperties());
@@ -146,13 +141,10 @@ public class TestGroupConstParallel {
         pc.defaultParallel = 100;
         pc.connect();
         
-        LogicalPlanTester planTester = new LogicalPlanTester() ;
-        planTester.buildPlan("a = load 'input';");
-        LogicalPlan lp = planTester.buildPlan("b = group a by $0;");
-        
-        PhysicalPlan pp = Util.getNewOptimizedPhysicalPlan(lp, pc);
-        POStore store = GenPhyOp.dummyPigStorageOp();
-        pp.addAsLeaf(store);
+        PigServer pigServer = new PigServer( ExecType.MAPREDUCE, cluster.getProperties() );
+        String query =  "a = load 'input';\n" + "b = group a by $0;" + "store b into 'output';";
+        
+        PhysicalPlan pp = Util.buildPp( pigServer, query );
         MROperPlan mrPlan = Util.buildMRPlan(pp, pc);
 
         ConfigurationValidator.validatePigProperties(pc.getProperties());