You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by th...@apache.org on 2011/05/23 22:32:20 UTC

svn commit: r1126723 - in /pig/trunk: ./ src/org/apache/pig/ src/org/apache/pig/tools/grunt/ test/org/apache/pig/parser/ test/org/apache/pig/test/

Author: thejas
Date: Mon May 23 20:32:19 2011
New Revision: 1126723

URL: http://svn.apache.org/viewvc?rev=1126723&view=rev
Log:
PIG-2084: pig is running validation for a statement at a time batch mode,
 instead of running it for whole script

Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/PigServer.java
    pig/trunk/src/org/apache/pig/tools/grunt/Grunt.java
    pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java
    pig/trunk/test/org/apache/pig/parser/TestErrorHandling.java
    pig/trunk/test/org/apache/pig/test/TestBuiltin.java
    pig/trunk/test/org/apache/pig/test/TestDataBagAccess.java
    pig/trunk/test/org/apache/pig/test/TestGrunt.java
    pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java
    pig/trunk/test/org/apache/pig/test/TestNewPlanLogToPhyTranslationVisitor.java
    pig/trunk/test/org/apache/pig/test/TestPigServer.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1126723&r1=1126722&r2=1126723&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Mon May 23 20:32:19 2011
@@ -231,6 +231,9 @@ PIG-1696: Performance: Use System.arrayc
 
 BUG FIXES
 
+PIG-2084: pig is running validation for a statement at a time batch mode, 
+ instead of running it for whole script (thejas)
+
 PIG-2088: Return alias validation failed when there is single line comment in the macro (rding)
 
 PIG-2081: Dryrun gives wrong line numbers in error message for scripts containing macro (rding)

Modified: pig/trunk/src/org/apache/pig/PigServer.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/PigServer.java?rev=1126723&r1=1126722&r2=1126723&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/PigServer.java (original)
+++ pig/trunk/src/org/apache/pig/PigServer.java Mon May 23 20:32:19 2011
@@ -183,6 +183,8 @@ public class PigServer {
     private boolean isMultiQuery = true;
     private boolean aggregateWarning = true;
 
+    private boolean validateEachStatement = false;
+
     private String constructScope() {
         // scope servers for now as a session id
 
@@ -578,7 +580,7 @@ public class PigServer {
      * @throws IOException
      */
     public void registerQuery(String query, int startLine) throws IOException {
-        currDAG.registerQuery(query, startLine);
+        currDAG.registerQuery(query, startLine, validateEachStatement);
     }
 
     /**
@@ -1566,7 +1568,8 @@ public class PigServer {
          * Accumulate the given statement to previous query statements and generate
          * an overall (raw) plan.
          */
-        void registerQuery(String query, int startLine) throws IOException {
+        void registerQuery(String query, int startLine, boolean validateEachStatement)
+        throws IOException {
             if( batchMode ) {
                 if( startLine == currentLineNum ) {
                     String line = scriptCache.remove( scriptCache.size() - 1 );
@@ -1588,7 +1591,9 @@ public class PigServer {
                 scriptCache.add( query );
             }
            
-            validateQuery();
+            if(validateEachStatement){
+                validateQuery();
+            }
             parseQuery();
             
             if( !batchMode ) {
@@ -1763,10 +1768,11 @@ public class PigServer {
 
             try {
                 for (Iterator<String> it = scriptCache.iterator(); it.hasNext(); lineNumber++) {
-                	// always doing registerQuery irrespective of the batch mode
-                	// TODO: Need to figure out if anything different needs to happen if batch 
-                	// mode is not on
-                    graph.registerQuery(it.next(), lineNumber);
+                    // always doing registerQuery irrespective of the batch mode
+                    // TODO: Need to figure out if anything different needs to happen if batch 
+                    // mode is not on
+                    // Don't have to do the validation again, so set validateEachStatement param to false
+                    graph.registerQuery(it.next(), lineNumber, false);
                 }
                 graph.postProcess();
             } catch (IOException ioe) {
@@ -1776,4 +1782,14 @@ public class PigServer {
             return graph;
         }
     }
+
+    /**
+     * This can be called to indicate if the query is being parsed/compiled
+     * in a mode that expects each statement to be validated as it is 
+     * entered, instead of just doing it once for whole script.
+     * @param validateEachStatement
+     */
+    public void setValidateEachStatement(boolean validateEachStatement) {
+        this.validateEachStatement = validateEachStatement;
+    }
 }

Modified: pig/trunk/src/org/apache/pig/tools/grunt/Grunt.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/tools/grunt/Grunt.java?rev=1126723&r1=1126722&r2=1126723&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/tools/grunt/Grunt.java (original)
+++ pig/trunk/src/org/apache/pig/tools/grunt/Grunt.java Mon May 23 20:32:19 2011
@@ -90,6 +90,7 @@ public class Grunt 
         boolean verbose = "true".equalsIgnoreCase(pig.getPigContext().getProperties().getProperty("verbose"));
         try {
             parser.setInteractive(false);
+            parser.setValidateEachStatement(true);
             boolean dontPrintOutput = true;
             parser.processExplain(null, scriptFile, false, "text", null, 
                     new ArrayList<String>(), new ArrayList<String>(),

Modified: pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java?rev=1126723&r1=1126722&r2=1126723&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java (original)
+++ pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java Mon May 23 20:32:19 2011
@@ -105,6 +105,18 @@ public class GruntParser extends PigScri
         mScriptIllustrate = false;
     }
 
+    @Override
+    public void setInteractive(boolean isInteractive){
+        super.setInteractive(isInteractive);
+        if(isInteractive){
+            setValidateEachStatement(true);
+        }
+    }
+    
+    public void setValidateEachStatement(boolean b) {
+        mPigServer.setValidateEachStatement(b);
+    }
+
     private void setBatchOn() {
         mPigServer.setBatchOn();
     }

Modified: pig/trunk/test/org/apache/pig/parser/TestErrorHandling.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/parser/TestErrorHandling.java?rev=1126723&r1=1126722&r2=1126723&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/parser/TestErrorHandling.java (original)
+++ pig/trunk/test/org/apache/pig/parser/TestErrorHandling.java Mon May 23 20:32:19 2011
@@ -36,6 +36,7 @@ public class TestErrorHandling {
     @Before
     public void setUp() throws Exception{
         pig = new PigServer(ExecType.LOCAL, new Properties());
+        pig.setValidateEachStatement(true);
     }
 
     @Test // Error from SchemaAliasVisitor

Modified: pig/trunk/test/org/apache/pig/test/TestBuiltin.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestBuiltin.java?rev=1126723&r1=1126722&r2=1126723&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestBuiltin.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestBuiltin.java Mon May 23 20:32:19 2011
@@ -157,6 +157,7 @@ public class TestBuiltin {
     public void setUp() throws Exception {
        
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        pigServer.setValidateEachStatement(true);
         // First set up data structs for "base" SUM, MIN and MAX and AVG.
         // The allowed input and expected output data structs for 
         // the "Intermediate" and "Final" stages can be based on the 

Modified: pig/trunk/test/org/apache/pig/test/TestDataBagAccess.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestDataBagAccess.java?rev=1126723&r1=1126722&r2=1126723&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestDataBagAccess.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestDataBagAccess.java Mon May 23 20:32:19 2011
@@ -120,6 +120,7 @@ public class TestDataBagAccess extends T
         File input = Util.createInputFile("tmp", "", 
                 new String[] {"sampledata\tnot_used"});
         boolean exceptionOccured = false;
+        pigServer.setValidateEachStatement(true);
         try {
             pigServer.registerQuery("a = load '" 
                     + Util.generateURI(Util.encodeEscape(input.toString()), pigServer.getPigContext()) + "';");

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=1126723&r1=1126722&r2=1126723&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestGrunt.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestGrunt.java Mon May 23 20:32:19 2011
@@ -33,18 +33,29 @@ import org.apache.pig.PigException;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.impl.PigContext;
+import org.apache.pig.test.Util.ProcessReturnInfo;
 import org.apache.pig.tools.grunt.Grunt;
+import org.apache.pig.tools.parameters.ParameterSubstitutionPreprocessor;
 import org.apache.pig.tools.pigscript.parser.ParseException;
+import org.apache.pig.tools.pigstats.ScriptState;
 import org.apache.pig.impl.io.FileLocalizer;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 
 import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.FileReader;
+import java.io.FileWriter;
 import java.io.InputStreamReader;
 import java.io.BufferedReader;
+import java.io.PrintWriter;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Properties;
 
 @RunWith(JUnit4.class)
 public class TestGrunt extends TestCase {
+ 
     static MiniCluster cluster = MiniCluster.buildCluster();
     private String basedir = "test/org/apache/pig/test/data";
 
@@ -481,6 +492,33 @@ public class TestGrunt extends TestCase 
     
         grunt.exec();
     }
+    
+    /**
+     * PIG-2084
+     * Check if only statements used in query are validated, in non-interactive
+     * /non-check mode. There is an  'unused' statement in query that would otherise
+     * fail the validation. 
+     * Primary purpose of test is to verify that check not happening for 
+     *  every statement. 
+     * @throws Throwable
+     */
+    @Test
+    public void testExplainScriptIsEachStatementValidated() throws Throwable {
+        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigContext context = server.getPigContext();
+        
+        String strCmd = "a = load 'foo' as (foo, fast, regenerate);" +
+        		"b = foreach a generate foo + 'x' + 1;" +
+        		"c = foreach a generate foo, fast;" +
+        		"explain c; ";
+        
+        ByteArrayInputStream cmd = new ByteArrayInputStream(strCmd.getBytes());
+        InputStreamReader reader = new InputStreamReader(cmd);
+        
+        Grunt grunt = new Grunt(new BufferedReader(reader), context);
+    
+        grunt.exec();
+    }
 
     @Test
     public void testIllustrateScript() throws Throwable {
@@ -1136,7 +1174,7 @@ public class TestGrunt extends TestCase 
         assertTrue(context.getFuncSpecFromAlias("pig.square") != null);
 
     }
-    /*
+    
     @Test    
     public void testScriptMissingLastNewLine() throws Throwable {   
         PigServer server = new PigServer(ExecType.LOCAL);
@@ -1157,8 +1195,10 @@ public class TestGrunt extends TestCase 
         }
     }
     
+    /*
     // Test case for PIG-740 to report an error near the double quotes rather
     // than an unrelated EOF error message
+    @Test
     public void testBlockErrMessage() throws Throwable {
         PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
         PigContext context = server.getPigContext();
@@ -1179,6 +1219,7 @@ public class TestGrunt extends TestCase 
         }
     }
     
+    @Test
     public void testCheckScript() throws Throwable {
         // a query which has grunt commands intermixed with pig statements - this
         // should pass through successfully with the check and all the grunt commands
@@ -1202,6 +1243,7 @@ public class TestGrunt extends TestCase 
         validate(query, true, msgs.toArray(new String[0]));
     }
     
+    @Test
     public void testCheckScriptSyntaxErr() throws Throwable {
         // a query which has grunt commands intermixed with pig statements - this
         // should fail with the -check option with a syntax error
@@ -1221,6 +1263,7 @@ public class TestGrunt extends TestCase 
         validate(query, false, msgs.toArray(new String[0]));
     }
     
+    @Test
     public void testCheckScriptTypeCheckErr() throws Throwable {
         // a query which has grunt commands intermixed with pig statements - this
         // should fail with the -check option with a type checking error
@@ -1240,6 +1283,8 @@ public class TestGrunt extends TestCase 
         validate(query, false, msgs.toArray(new String[0]));
     }
     
+
+    
     private void validate(String query, boolean syntaxOk, 
             String[] logMessagesToCheck) throws Throwable {
         File scriptFile = Util.createFile(new String[] { query});
@@ -1262,15 +1307,42 @@ public class TestGrunt extends TestCase 
                     pri.stderrContents.contains("syntax OK"));
         }
     }
-    
+    */
     @Test
     public void testSet() throws Throwable {
 
-    	String strCmd = "set my.arbitrary.key my.arbitrary.value\n";
-    	PigContext pc = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()).getPigContext();
-    	InputStreamReader reader = new InputStreamReader(new ByteArrayInputStream(strCmd.getBytes()));
+        String strCmd = "set my.arbitrary.key my.arbitrary.value\n";
+        PigContext pc = new PigServer(ExecType.LOCAL).getPigContext();
+        InputStreamReader reader = new InputStreamReader(new ByteArrayInputStream(strCmd.getBytes()));
         new Grunt(new BufferedReader(reader), pc).exec();
-        
-        assertEquals("my.arbitrary.value",  pc.getExecutionEngine().getConfiguration().getProperty("my.arbitrary.key"));
-    }*/
-}
+
+        assertEquals("my.arbitrary.value",  pc.getProperties().getProperty("my.arbitrary.key"));
+    }
+
+    @Test
+    public void testCheckScriptTypeCheckErrNoStoreDump() throws Throwable {
+        //the query has not store or dump, but in when -check is used
+        // all statements should be validated
+        String query = "a = load 'foo.pig' as (s:chararray); " +
+        "b = foreach a generate $1;";
+
+        String msg = "Trying to access non-existent column";
+        validateGruntCheckFail(query, msg);
+    }
+
+    private void validateGruntCheckFail(String piglatin, String errMsg) throws Throwable{
+        String scriptFile = "myscript.pig";
+        try {
+            BufferedReader br = new BufferedReader(new StringReader(piglatin));
+            Grunt grunt = new Grunt(br, new PigContext(ExecType.LOCAL, new Properties()));
+            String [] inp = {piglatin};
+            Util.createLocalInputFile(scriptFile, inp);
+
+            grunt.checkScript(scriptFile);
+
+            Assert.fail("Expected exception isn't thrown");
+        } catch (FrontendException e) { 
+            Util.checkMessageInException(e, errMsg);
+        }
+    }
+}
\ No newline at end of file

Modified: pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java?rev=1126723&r1=1126722&r2=1126723&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java Mon May 23 20:32:19 2011
@@ -36,6 +36,7 @@ import org.apache.hadoop.mapreduce.Recor
 import org.apache.pig.LoadCaster;
 import org.apache.pig.LoadFunc;
 import org.apache.pig.FuncSpec;
+import org.apache.pig.PigException;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit;
 import org.apache.pig.builtin.PigStorage;
@@ -47,6 +48,7 @@ import org.apache.pig.impl.PigContext;
 import org.apache.pig.ExecType;
 import org.apache.pig.impl.builtin.GFAny;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.impl.util.LogUtils;
 import org.apache.pig.impl.util.Utils;
 import org.apache.pig.data.DataType;
 import org.apache.pig.newplan.Operator;
@@ -2104,9 +2106,15 @@ public class TestLogicalPlanBuilder {
     	try {
             return Util.buildLp(pigServer, query);
     	} catch(Throwable t) {
-    		Throwable cause = t.getCause();
-    		String msg = cause != null ? cause.toString() : t.toString();
-    		throw new AssertionFailedError( msg );
+    	    PigException pigEx = LogUtils.getPigException(t);
+    	    Throwable cause = null;
+    	    if(pigEx != null){
+    	        cause = pigEx;
+    	    }else{
+    	        cause = t.getCause();
+    	    }
+    	    String msg = cause != null ? cause.toString() : t.toString();
+    	    throw new AssertionFailedError( msg );
     	}
     }
     

Modified: pig/trunk/test/org/apache/pig/test/TestNewPlanLogToPhyTranslationVisitor.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestNewPlanLogToPhyTranslationVisitor.java?rev=1126723&r1=1126722&r2=1126723&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestNewPlanLogToPhyTranslationVisitor.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestNewPlanLogToPhyTranslationVisitor.java Mon May 23 20:32:19 2011
@@ -508,9 +508,9 @@ public class TestNewPlanLogToPhyTranslat
         LogicalRelationalOperator ld =  (LogicalRelationalOperator)newLogicalPlan.getSources().get(0);
         LogicalRelationalOperator fe = (LogicalRelationalOperator)newLogicalPlan.getSuccessors(ld).get(0);
         LogicalSchema ls = fe.getSchema();
-        assertEquals(6, ls.getField(0).uid);
-        assertEquals(9, ls.getField(1).uid);
-        assertEquals(10, ls.getField(2).uid);
+        assertEquals(1, ls.getField(0).uid);
+        assertEquals(4, ls.getField(1).uid);
+        assertEquals(5, ls.getField(2).uid);
         
         LogicalSchema expected = new LogicalSchema();
         expected.addField(new LogicalFieldSchema("id", null, DataType.BYTEARRAY));
@@ -561,8 +561,8 @@ public class TestNewPlanLogToPhyTranslat
         POForEach pForEach = (POForEach)pFE;
         PhysicalPlan inputPln = pForEach.getInputPlans().get(0);
         
-        assertEquals(4, ls.getField(0).uid);
-        assertEquals(5, ls.getField(1).uid);
+        assertEquals(1, ls.getField(0).uid);
+        assertEquals(2, ls.getField(1).uid);
         
         LogicalRelationalOperator fe = 
             (LogicalRelationalOperator) newLogicalPlan.getSuccessors(load).get(0);
@@ -615,8 +615,8 @@ public class TestNewPlanLogToPhyTranslat
         POForEach pForEach = (POForEach)pFE;
         PhysicalPlan inputPln = pForEach.getInputPlans().get(0);
         
-        assertEquals(4, ls.getField(0).uid);
-        assertEquals(5, ls.getField(1).uid);
+        assertEquals(1, ls.getField(0).uid);
+        assertEquals(2, ls.getField(1).uid);
         
         LogicalRelationalOperator fe = 
             (LogicalRelationalOperator) newLogicalPlan.getSuccessors(load).get(0);
@@ -668,8 +668,8 @@ public class TestNewPlanLogToPhyTranslat
         POForEach pForEach = (POForEach)pFE;
         PhysicalPlan inputPln = pForEach.getInputPlans().get(0);
         
-        assertEquals(4, ls.getField(0).uid);
-        assertEquals(5, ls.getField(1).uid);
+        assertEquals(1, ls.getField(0).uid);
+        assertEquals(2, ls.getField(1).uid);
         
         LogicalRelationalOperator fe = 
             (LogicalRelationalOperator) newLogicalPlan.getSuccessors(load).get(0);
@@ -721,8 +721,8 @@ public class TestNewPlanLogToPhyTranslat
         POForEach pForEach = (POForEach)pFE;
         PhysicalPlan inputPln = pForEach.getInputPlans().get(0);
         
-        assertEquals(4, ls.getField(0).uid);
-        assertEquals(5, ls.getField(1).uid);
+        assertEquals(1, ls.getField(0).uid);
+        assertEquals(2, ls.getField(1).uid);
         
         LogicalRelationalOperator fe = 
             (LogicalRelationalOperator) newLogicalPlan.getSuccessors(load).get(0);
@@ -774,8 +774,8 @@ public class TestNewPlanLogToPhyTranslat
         POForEach pForEach = (POForEach)pFE;
         PhysicalPlan inputPln = pForEach.getInputPlans().get(0);
         
-        assertEquals(4, ls.getField(0).uid);
-        assertEquals(5, ls.getField(1).uid);
+        assertEquals(1, ls.getField(0).uid);
+        assertEquals(2, ls.getField(1).uid);
         
         LogicalRelationalOperator fe = 
             (LogicalRelationalOperator) newLogicalPlan.getSuccessors(load).get(0);
@@ -827,8 +827,8 @@ public class TestNewPlanLogToPhyTranslat
         POForEach pForEach = (POForEach)pFE;
         PhysicalPlan inputPln = pForEach.getInputPlans().get(0);
         
-        assertEquals(4, ls.getField(0).uid);
-        assertEquals(5, ls.getField(1).uid);
+        assertEquals(1, ls.getField(0).uid);
+        assertEquals(2, ls.getField(1).uid);
         
         LogicalRelationalOperator fe = 
             (LogicalRelationalOperator) newLogicalPlan.getSuccessors(load).get(0);
@@ -871,8 +871,8 @@ public class TestNewPlanLogToPhyTranslat
         LOLoad load = (LOLoad)ld;
         LogicalSchema ls = load.getSchema();
         
-        assertEquals(4, ls.getField(0).uid);
-        assertEquals(5, ls.getField(1).uid);
+        assertEquals(1, ls.getField(0).uid);
+        assertEquals(2, ls.getField(1).uid);
         
         LogicalRelationalOperator fil = (LogicalRelationalOperator)
         newLogicalPlan.getSuccessors( newLogicalPlan.getSources().get(0) ).get(0);
@@ -906,8 +906,8 @@ public class TestNewPlanLogToPhyTranslat
         LOLoad load = (LOLoad)ld;
         LogicalSchema ls = load.getSchema();
         
-        assertEquals(5, ls.getField(0).uid);
-        assertEquals(6, ls.getField(1).uid);
+        assertEquals(1, ls.getField(0).uid);
+        assertEquals(2, ls.getField(1).uid);
         
         LogicalRelationalOperator fil = (LogicalRelationalOperator)
         newLogicalPlan.getSuccessors( newLogicalPlan.getSources().get(0) ).get(0);
@@ -952,8 +952,8 @@ public class TestNewPlanLogToPhyTranslat
         POForEach pForEach = (POForEach)pFE;
         PhysicalPlan inputPln = pForEach.getInputPlans().get(0);
         
-        assertEquals(5, ls.getField(0).uid);
-        assertEquals(6, ls.getField(1).uid);
+        assertEquals(1, ls.getField(0).uid);
+        assertEquals(2, ls.getField(1).uid);
         
         LogicalRelationalOperator fe = 
             (LogicalRelationalOperator) newLogicalPlan.getSuccessors(load).get(0);
@@ -1033,8 +1033,8 @@ public class TestNewPlanLogToPhyTranslat
         POForEach pForEach = (POForEach)pFE;
         PhysicalPlan inputPln1 = pForEach.getInputPlans().get(0);
         
-        assertEquals(7, ls.getField(0).uid);
-        assertEquals(8, ls.getField(1).uid);
+        assertEquals(1, ls.getField(0).uid);
+        assertEquals(2, ls.getField(1).uid);
         
         LogicalRelationalOperator fe = 
             (LogicalRelationalOperator) newLogicalPlan.getSuccessors(load).get(0);
@@ -1094,8 +1094,8 @@ public class TestNewPlanLogToPhyTranslat
         POForEach pForEach = (POForEach)pFE;
         PhysicalPlan inputPln = pForEach.getInputPlans().get(0);
         
-        assertEquals(11, ls.getField(0).uid);
-        assertEquals(12, ls.getField(1).uid);
+        assertEquals(1, ls.getField(0).uid);
+        assertEquals(2, ls.getField(1).uid);
         
         LogicalRelationalOperator fe = 
             (LogicalRelationalOperator) newLogicalPlan.getSuccessors(load).get(0);

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=1126723&r1=1126722&r2=1126723&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPigServer.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPigServer.java Mon May 23 20:32:19 2011
@@ -49,7 +49,9 @@ import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.io.FileLocalizer;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.impl.util.LogUtils;
 import org.apache.pig.impl.util.PropertiesUtil;
 import org.apache.pig.impl.util.Utils;
 import org.junit.After;
@@ -780,13 +782,15 @@ public class TestPigServer {
 
     @Test // PIG-2059
     public void test1() throws Throwable {
-    	pig.setBatchOn();
+    	pig.setValidateEachStatement(true);
         pig.registerQuery("A = load 'x' as (u, v);") ;
         try {
             pig.registerQuery("B = foreach A generate $2;") ;
-        } catch(Exception ex) {
-        	Assert.assertTrue( ex.getMessage().contains("<line 1, column 46> Out of bound access. Trying to access non-existent column: 2") );
-        	return;
+        } catch(FrontendException ex) {
+            String msg = "Out of bound access. " +
+            "Trying to access non-existent column: 2";
+            Util.checkMessageInException(ex, msg);
+            return;
         }
         Assert.fail( "Query is supposed to fail." );
     }