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

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

Author: cheolsoo
Date: Tue Mar  5 17:25:46 2013
New Revision: 1452909

URL: http://svn.apache.org/r1452909
Log:
PIG-3136: Introduce a syntax making declared aliases optional (jcoveney via cheolsoo)

Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/PigServer.java
    pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java
    pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g
    pig/trunk/src/org/apache/pig/parser/QueryLexer.g
    pig/trunk/src/org/apache/pig/parser/QueryParser.g
    pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java
    pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java
    pig/trunk/src/org/apache/pig/tools/pigscript/parser/PigScriptParser.jj
    pig/trunk/test/org/apache/pig/test/TestBuiltin.java
    pig/trunk/test/org/apache/pig/test/TestShortcuts.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1452909&r1=1452908&r2=1452909&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Tue Mar  5 17:25:46 2013
@@ -28,6 +28,8 @@ PIG-3174:  Remove rpm and deb artifacts 
 
 IMPROVEMENTS
 
+PIG-3136: Introduce a syntax making declared aliases optional (jcoveney via cheolsoo)
+
 PIG-3142: [piggybank] Fixed-width load and store functions for the Piggybank (jpacker via cheolsoo)
 
 PIG-3162: PigTest.assertOutput doesn't allow non-default delimiter (dreambird via cheolsoo)

Modified: pig/trunk/src/org/apache/pig/PigServer.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/PigServer.java?rev=1452909&r1=1452908&r2=1452909&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/PigServer.java (original)
+++ pig/trunk/src/org/apache/pig/PigServer.java Tue Mar  5 17:25:46 2013
@@ -196,7 +196,7 @@ public class PigServer {
     public PigServer(ExecType execType, Properties properties) throws ExecException {
         this(new PigContext(execType, properties));
     }
-    
+
     public PigServer(ExecType execType, Configuration conf) throws ExecException {
         this(new PigContext(execType, conf));
     }
@@ -765,6 +765,9 @@ public class PigServer {
      */
     public Schema dumpSchema(String alias) throws IOException {
         try {
+            if ("@".equals(alias)) {
+                alias = getLastRel();
+            }
             LogicalRelationalOperator op = getOperatorForAlias( alias );
             LogicalSchema schema = op.getSchema();
 
@@ -796,6 +799,9 @@ public class PigServer {
      * @throws IOException
      */
     public Schema dumpSchemaNested(String alias, String nestedAlias) throws IOException {
+        if ("@".equals(alias)) {
+            alias = getLastRel();
+        }
         Operator op = getOperatorForAlias( alias );
         if( op instanceof LOForEach ) {
             LogicalSchema nestedSc = ((LOForEach)op).dumpNestedSchema(alias, nestedAlias);
@@ -969,6 +975,9 @@ public class PigServer {
 
     private PigStats storeEx(String alias, String filename, String func)
     throws IOException {
+        if ("@".equals(alias)) {
+            alias = getLastRel();
+        }
         currDAG.parseQuery();
         currDAG.buildPlan( alias );
 
@@ -1237,14 +1246,14 @@ public class PigServer {
 
     public void printHistory(boolean withNumbers) {
 
-    	List<String> sc = currDAG.getScriptCache();
+        List<String> sc = currDAG.getScriptCache();
 
-    	if(!sc.isEmpty()) {
-    		for(int i = 0 ; i < sc.size(); i++) {
-    			if(withNumbers) System.out.print((i+1)+"   ");
-    			System.out.println(sc.get(i));
-    		}
-    	}
+        if(!sc.isEmpty()) {
+            for(int i = 0 ; i < sc.size(); i++) {
+                if(withNumbers) System.out.print((i+1)+"   ");
+                System.out.println(sc.get(i));
+            }
+        }
 
     }
 
@@ -1389,6 +1398,7 @@ public class PigServer {
         private final Map<LogicalRelationalOperator, LogicalPlan> aliases = new HashMap<LogicalRelationalOperator, LogicalPlan>();
 
         private Map<String, Operator> operators = new HashMap<String, Operator>();
+        private String lastRel;
 
         private final List<String> scriptCache = new ArrayList<String>();
 
@@ -1628,7 +1638,7 @@ public class PigServer {
         }
 
         public List<String> getScriptCache() {
-        	return scriptCache;
+            return scriptCache;
         }
 
         /**
@@ -1649,6 +1659,7 @@ public class PigServer {
                 QueryParserDriver parserDriver = new QueryParserDriver( pigContext, scope, fileNameMap );
                 lp = parserDriver.parse( query );
                 operators = parserDriver.getOperators();
+                lastRel = parserDriver.getLastRel();
             } catch(Exception ex) {
                 scriptCache.remove( scriptCache.size() -1 ); // remove the bad script from the cache.
                 PigException pe = LogUtils.getPigException(ex);
@@ -1659,6 +1670,10 @@ public class PigServer {
             }
         }
 
+        public String getLastRel() {
+            return lastRel;
+        }
+
         private String buildQuery() {
             StringBuilder accuQuery = new StringBuilder();
             for( String line : scriptCache ) {
@@ -1816,4 +1831,8 @@ public class PigServer {
     public void setValidateEachStatement(boolean validateEachStatement) {
         this.validateEachStatement = validateEachStatement;
     }
+
+    public String getLastRel() {
+        return currDAG.getLastRel();
+    }
 }

Modified: pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java?rev=1452909&r1=1452908&r2=1452909&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java (original)
+++ pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java Tue Mar  5 17:25:46 2013
@@ -1658,4 +1658,8 @@ public class LogicalPlanBuilder {
       return lastRel;
   }
 
+    public String getLastRel() {
+        return lastRel;
+    }
+
 }

Modified: pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g?rev=1452909&r1=1452908&r2=1452909&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g (original)
+++ pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g Tue Mar  5 17:25:46 2013
@@ -120,6 +120,10 @@ public Map<String, Operator> getOperator
     return builder.getOperators();
 }
 
+public String getLastRel() {
+    return builder.getLastRel();
+}
+
 @Override
 protected Object recoverFromMismatchedToken(IntStream input, int ttype, BitSet follow)
 throws RecognitionException {

Modified: pig/trunk/src/org/apache/pig/parser/QueryLexer.g
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/QueryLexer.g?rev=1452909&r1=1452908&r2=1452909&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/QueryLexer.g (original)
+++ pig/trunk/src/org/apache/pig/parser/QueryLexer.g Tue Mar  5 17:25:46 2013
@@ -450,3 +450,6 @@ QMARK : '?'
 
 ARROBA : '@'
 ;
+
+FAT_ARROW : '=>'
+;

Modified: pig/trunk/src/org/apache/pig/parser/QueryParser.g
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/QueryParser.g?rev=1452909&r1=1452908&r2=1452909&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/QueryParser.g (original)
+++ pig/trunk/src/org/apache/pig/parser/QueryParser.g Tue Mar  5 17:25:46 2013
@@ -75,6 +75,7 @@ tokens {
     TOBAG;
     TOMAP;
     TOTUPLE;
+    FAT_ARROW;
 }
 
 @header {
@@ -209,14 +210,17 @@ nested_op_clause : LEFT_PAREN! op_clause
                     -> ^( FOREACH rel foreach_plan_complex? foreach_plan_simple? ) parallel_clause?
 ;
 
-general_statement : ( IDENTIFIER EQUAL )? ( ( op_clause parallel_clause? ) | nested_op_clause ) -> ^( STATEMENT IDENTIFIER? op_clause? parallel_clause? nested_op_clause? )
+general_statement : FAT_ARROW ( ( op_clause parallel_clause? ) | nested_op_clause ) -> ^( STATEMENT IDENTIFIER["____RESERVED____"] op_clause? parallel_clause? nested_op_clause? )
+                  | ( IDENTIFIER EQUAL )? ( ( op_clause parallel_clause? ) | nested_op_clause ) -> ^( STATEMENT IDENTIFIER? op_clause? parallel_clause? nested_op_clause? )
 ;
 
 // Statement represented by a foreach operator with a nested block. Simple foreach statement
 // is covered by general_statement.
 // We need to handle foreach specifically because of the ending ';', which is not required
 // if there is a nested block. This is ugly, but it gets the job done.
-foreach_statement : ( IDENTIFIER EQUAL )? FOREACH rel ( foreach_plan_complex | ( foreach_plan_simple parallel_clause? SEMI_COLON ) )
+foreach_statement : FAT_ARROW FOREACH rel ( foreach_plan_complex | ( foreach_plan_simple parallel_clause? SEMI_COLON ) )
+    -> ^( STATEMENT IDENTIFIER["____RESERVED____"] ^( FOREACH rel foreach_plan_complex? foreach_plan_simple? ) parallel_clause? )
+                  | ( IDENTIFIER EQUAL )? FOREACH rel ( foreach_plan_complex | ( foreach_plan_simple parallel_clause? SEMI_COLON ) )
     -> ^( STATEMENT IDENTIFIER? ^( FOREACH rel foreach_plan_complex? foreach_plan_simple? ) parallel_clause? )
 ;
 

Modified: pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java?rev=1452909&r1=1452908&r2=1452909&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java (original)
+++ pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java Tue Mar  5 17:25:46 2013
@@ -19,13 +19,10 @@
 package org.apache.pig.parser;
 
 import java.io.BufferedReader;
-import java.io.FileReader;
 import java.io.File;
 import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
+import java.io.FileReader;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -42,7 +39,6 @@ import org.antlr.runtime.tree.CommonTree
 import org.antlr.runtime.tree.Tree;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.Path;
 import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.impl.PigContext;
@@ -67,6 +63,7 @@ public class QueryParserDriver {
     private String scope;
     private Map<String, String>fileNameMap;
     private Map<String, Operator> operators;
+    private String lastRel;
     private Set<String> importSeen;
     private Set<String> macroSeen;
 
@@ -187,6 +184,7 @@ public class QueryParserDriver {
 
             plan = planGenerator.getLogicalPlan();
             operators = planGenerator.getOperators();
+            lastRel = planGenerator.getLastRel();
         } catch(RecognitionException ex) {
             throw new ParserException( ex );
         } catch(Exception ex) {
@@ -538,4 +536,8 @@ public class QueryParserDriver {
         }
         return sb.toString();
     }
+
+    public String getLastRel() {
+        return lastRel;
+    }
 }

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=1452909&r1=1452908&r2=1452909&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java (original)
+++ pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java Tue Mar  5 17:25:46 2013
@@ -118,7 +118,7 @@ public class GruntParser extends PigScri
             setValidateEachStatement(true);
         }
     }
-    
+
     public void setValidateEachStatement(boolean b) {
         mPigServer.setValidateEachStatement(b);
     }
@@ -145,10 +145,10 @@ public class GruntParser extends PigScri
                         Exception exp = (js.getException() != null) ? js.getException()
                                 : new ExecException(
                                         "Job failed, hadoop does not return any error message",
-                                        2244);                        
-                        LogUtils.writeLog(exp, 
-                                mPigServer.getPigContext().getProperties().getProperty("pig.logfile"), 
-                                log, 
+                                        2244);
+                        LogUtils.writeLog(exp,
+                                mPigServer.getPigContext().getProperties().getProperty("pig.logfile"),
+                                log,
                                 "true".equalsIgnoreCase(mPigServer.getPigContext().getProperties().getProperty("verbose")),
                                 "Pig Stack Trace");
                     } else {
@@ -167,12 +167,12 @@ public class GruntParser extends PigScri
 
     public int[] parseStopOnError() throws IOException, ParseException
     {
-	return parseStopOnError(false);
+        return parseStopOnError(false);
     }
-    
-    /** 
-     * Parses Pig commands in either interactive mode or batch mode. 
-     * In interactive mode, executes the plan right away whenever a 
+
+    /**
+     * Parses Pig commands in either interactive mode or batch mode.
+     * In interactive mode, executes the plan right away whenever a
      * STORE command is encountered.
      *
      * @throws IOException, ParseException
@@ -193,7 +193,7 @@ public class GruntParser extends PigScri
             while(!mDone) {
                 parse();
             }
-            
+
             if (!sameBatch) {
                 executeBatch();
             }
@@ -206,7 +206,7 @@ public class GruntParser extends PigScri
         return res;
     }
 
-    public void setLoadOnly(boolean loadOnly) 
+    public void setLoadOnly(boolean loadOnly)
     {
         mLoadOnly = loadOnly;
     }
@@ -214,16 +214,16 @@ public class GruntParser extends PigScri
     public void setParams(PigServer pigServer)
     {
         mPigServer = pigServer;
-        
+
         mDfs = mPigServer.getPigContext().getDfs();
         mLfs = mPigServer.getPigContext().getLfs();
         mConf = mPigServer.getPigContext().getProperties();
         shell = new FsShell(ConfigurationUtil.toConfiguration(mConf));
-        
+
         // TODO: this violates the abstraction layer decoupling between
         // front end and back end and needs to be changed.
         // Right now I am not clear on how the Job Id comes from to tell
-        // the back end to kill a given job (mJobClient is used only in 
+        // the back end to kill a given job (mJobClient is used only in
         // processKill)
         //
         HExecutionEngine execEngine = mPigServer.getPigContext().getExecutionEngine();
@@ -233,7 +233,7 @@ public class GruntParser extends PigScri
     public void setScriptIllustrate() {
         mScriptIllustrate = true;
     }
-    
+
     @Override
     public void prompt()
     {
@@ -241,7 +241,7 @@ public class GruntParser extends PigScri
             mConsoleReader.setDefaultPrompt("grunt> ");
         }
     }
-    
+
     @Override
     protected void quit()
     {
@@ -264,8 +264,8 @@ public class GruntParser extends PigScri
         while(!mDone) {
             parse();
         }
-    }    
-    
+    }
+
     @Override
     protected void processDescribe(String alias) throws IOException {
         String nestedAlias = null;
@@ -285,6 +285,9 @@ public class GruntParser extends PigScri
                 mPigServer.dumpSchemaNested(alias, nestedAlias);
             }
             else {
+                if ("@".equals(alias)) {
+                    alias = mPigServer.getLastRel();
+                }
                 mPigServer.dumpSchema(alias);
             }
         } else {
@@ -293,9 +296,9 @@ public class GruntParser extends PigScri
     }
 
     @Override
-    protected void processExplain(String alias, String script, boolean isVerbose, 
-                                  String format, String target, 
-                                  List<String> params, List<String> files) 
+    protected void processExplain(String alias, String script, boolean isVerbose,
+                                  String format, String target,
+                                  List<String> params, List<String> files)
     throws IOException, ParseException {
         if (alias == null && script == null) {
             alias = mPigServer.getPigContext().getLastAlias();
@@ -305,23 +308,26 @@ public class GruntParser extends PigScri
                 throw new ParseException("'explain' statement must be on an alias or on a script.");
             }
         }
+        if ("@".equals(alias)) {
+            alias = mPigServer.getLastRel();
+        }
         processExplain(alias, script, isVerbose, format, target, params, files, 
                 false);
     }
 
-    protected void processExplain(String alias, String script, boolean isVerbose, 
-                                  String format, String target, 
+    protected void processExplain(String alias, String script, boolean isVerbose,
+                                  String format, String target,
                                   List<String> params, List<String> files,
-                                  boolean dontPrintOutput) 
+                                  boolean dontPrintOutput)
         throws IOException, ParseException {
-        
+
         if (null != mExplain) {
             return;
         }
 
         try {
             mExplain = new ExplainState(alias, target, script, isVerbose, format);
-            
+
             if (script != null) {
                 if (!"true".equalsIgnoreCase(mPigServer.
                                              getPigContext()
@@ -357,8 +363,8 @@ public class GruntParser extends PigScri
     }
 
     /**
-     * A {@link PrintStream} implementation which does not write anything 
-     * Used with '-check' command line option to pig Main 
+     * A {@link PrintStream} implementation which does not write anything
+     * Used with '-check' command line option to pig Main
      * (through {@link GruntParser#explainCurrentBatch(boolean) } )
      */
     static class NullPrintStream extends PrintStream {
@@ -372,12 +378,12 @@ public class GruntParser extends PigScri
         @Override
         public void write(byte [] b) {}
     }
-    
+
     protected void explainCurrentBatch(boolean dontPrintOutput) throws IOException {
         PrintStream lp = (dontPrintOutput) ? new NullPrintStream("dummy") : System.out;
         PrintStream pp = (dontPrintOutput) ? new NullPrintStream("dummy") : System.out;
         PrintStream ep = (dontPrintOutput) ? new NullPrintStream("dummy") : System.out;
-        
+
         if (!(mExplain.mLast && mExplain.mCount == 0)) {
             if (mPigServer.isBatchEmpty()) {
                 return;
@@ -389,13 +395,13 @@ public class GruntParser extends PigScri
 
         if (mExplain.mTarget != null) {
             File file = new File(mExplain.mTarget);
-            
+
             if (file.isDirectory()) {
                 String sCount = (mExplain.mLast && mExplain.mCount == 1)?"":"_"+mExplain.mCount;
                 lp = new PrintStream(new File(file, "logical_plan-"+mExplain.mTime+sCount+"."+mExplain.mFormat));
                 pp = new PrintStream(new File(file, "physical_plan-"+mExplain.mTime+sCount+"."+mExplain.mFormat));
                 ep = new PrintStream(new File(file, "exec_plan-"+mExplain.mTime+sCount+"."+mExplain.mFormat));
-                mPigServer.explain(mExplain.mAlias, mExplain.mFormat, 
+                mPigServer.explain(mExplain.mAlias, mExplain.mFormat,
                                    mExplain.mVerbose, markAsExecuted, lp, pp, ep);
                 lp.close();
                 pp.close();
@@ -404,13 +410,13 @@ public class GruntParser extends PigScri
             else {
                 boolean append = !(mExplain.mCount==1);
                 lp = pp = ep = new PrintStream(new FileOutputStream(mExplain.mTarget, append));
-                mPigServer.explain(mExplain.mAlias, mExplain.mFormat, 
+                mPigServer.explain(mExplain.mAlias, mExplain.mFormat,
                                    mExplain.mVerbose, markAsExecuted, lp, pp, ep);
                 lp.close();
             }
         }
         else {
-            mPigServer.explain(mExplain.mAlias, mExplain.mFormat, 
+            mPigServer.explain(mExplain.mAlias, mExplain.mFormat,
                                mExplain.mVerbose, markAsExecuted, lp, pp, ep);
         }
     }
@@ -425,7 +431,7 @@ public class GruntParser extends PigScri
     }
 
     @Override
-	protected void printClear() {
+    protected void printClear() {
         AnsiConsole.systemInstall();
         Ansi ansi = Ansi.ansi();
         System.out.println( ansi.eraseScreen() );
@@ -449,19 +455,19 @@ public class GruntParser extends PigScri
         else {
             mPigServer.registerCode(path, scriptingLang, namespace);
         }
-    }    
+    }
 
-    private String runPreprocessor(String script, List<String> params, 
-                                   List<String> files) 
+    private String runPreprocessor(String script, List<String> params,
+                                   List<String> files)
         throws IOException, ParseException {
 
         ParameterSubstitutionPreprocessor psp = new ParameterSubstitutionPreprocessor(50);
         StringWriter writer = new StringWriter();
 
         try{
-            psp.genSubstitutedFile(new BufferedReader(new FileReader(script)), 
-                                   writer,  
-                                   params.size() > 0 ? params.toArray(new String[0]) : null, 
+            psp.genSubstitutedFile(new BufferedReader(new FileReader(script)),
+                                   writer,
+                                   params.size() > 0 ? params.toArray(new String[0]) : null,
                                    files.size() > 0 ? files.toArray(new String[0]) : null);
         } catch (org.apache.pig.tools.parameters.ParseException pex) {
             throw new ParseException(pex.getMessage());
@@ -471,16 +477,16 @@ public class GruntParser extends PigScri
     }
 
     @Override
-    protected void processScript(String script, boolean batch, 
-                                 List<String> params, List<String> files) 
+    protected void processScript(String script, boolean batch,
+                                 List<String> params, List<String> files)
         throws IOException, ParseException {
-        
+
         if(mExplain == null) { // process only if not in "explain" mode
             if (script == null) {
                 executeBatch();
                 return;
             }
-            
+
             if (batch) {
                 setBatchOn();
                 mPigServer.setJobName(script);
@@ -499,13 +505,13 @@ public class GruntParser extends PigScri
     }
 
     private void loadScript(String script, boolean batch, boolean loadOnly, boolean illustrate,
-                            List<String> params, List<String> files) 
+                            List<String> params, List<String> files)
         throws IOException, ParseException {
-        
+
         Reader inputReader;
         ConsoleReader reader;
         boolean interactive;
-         
+
         try {
             FetchFileRet fetchFile = FileLocalizer.fetchFile(mConf, script);
             String cmds = runPreprocessor(fetchFile.file.getAbsolutePath(), params, files);
@@ -539,7 +545,7 @@ public class GruntParser extends PigScri
         if (illustrate)
             parser.setScriptIllustrate();
         parser.mExplain = mExplain;
-        
+
         parser.prompt();
         while(!parser.isDone()) {
             parser.parse();
@@ -573,8 +579,8 @@ public class GruntParser extends PigScri
             // Validate
             File file = new File(value);
             if (!file.exists() || file.isDirectory()) {
-                throw new IOException("Invalid value for stream.skippath:" + 
-                                      value); 
+                throw new IOException("Invalid value for stream.skippath:" +
+                                      value);
             }
             mPigServer.addPathToSkip(value);
         }
@@ -590,9 +596,9 @@ public class GruntParser extends PigScri
         {
             //mPigServer.getPigContext().getProperties().setProperty(key, value);
             // PIG-2508 properties need to be managed through JobConf
-            // since all other code depends on access to properties, 
-            // we need to re-populate from updated JobConf 
-            //java.util.HashSet<?> keysBefore = new java.util.HashSet<Object>(mPigServer.getPigContext().getProperties().keySet());        	
+            // since all other code depends on access to properties,
+            // we need to re-populate from updated JobConf
+            //java.util.HashSet<?> keysBefore = new java.util.HashSet<Object>(mPigServer.getPigContext().getProperties().keySet());
             // set current properties on jobConf
             Properties properties = mPigServer.getPigContext().getProperties();
             Configuration jobConf = mPigServer.getPigContext().getExecutionEngine().getJobConf();
@@ -612,43 +618,43 @@ public class GruntParser extends PigScri
             while (iter.hasNext()) {
                 Map.Entry<String, String> entry = iter.next();
                 properties.put(entry.getKey(), entry.getValue());
-            } 
+            }
             //keysBefore.removeAll(mPigServer.getPigContext().getProperties().keySet());
             //log.info("PIG-2508: keys dropped from properties: " + keysBefore);
         }
     }
-    
+
     @Override
     protected void processCat(String path) throws IOException
     {
         if(mExplain == null) { // process only if not in "explain" mode
-            
+
             executeBatch();
 
             try {
                 byte buffer[] = new byte[65536];
                 ElementDescriptor dfsPath = mDfs.asElement(path);
                 int rc;
-                
+
                 if (!dfsPath.exists())
                     throw new IOException("Directory " + path + " does not exist.");
-        
+
                 if (mDfs.isContainer(path)) {
                     ContainerDescriptor dfsDir = (ContainerDescriptor) dfsPath;
                     Iterator<ElementDescriptor> paths = dfsDir.iterator();
-                    
+
                     while (paths.hasNext()) {
                         ElementDescriptor curElem = paths.next();
-                        
+
                         if (mDfs.isContainer(curElem.toString())) {
                             continue;
                         }
-                        
+
                         InputStream is = curElem.open();
                         while ((rc = is.read(buffer)) > 0) {
                             System.out.write(buffer, 0, rc);
                         }
-                        is.close();                
+                        is.close();
                     }
                 }
                 else {
@@ -656,7 +662,7 @@ public class GruntParser extends PigScri
                     while ((rc = is.read(buffer)) > 0) {
                         System.out.write(buffer, 0, rc);
                     }
-                    is.close();            
+                    is.close();
                 }
             }
             catch (DataStorageException e) {
@@ -669,7 +675,7 @@ public class GruntParser extends PigScri
 
     @Override
     protected void processCD(String path) throws IOException
-    {    
+    {
         ContainerDescriptor container;
         if(mExplain == null) { // process only if not in "explain" mode
             try {
@@ -680,21 +686,21 @@ public class GruntParser extends PigScri
                 else
                 {
                     container = mDfs.asContainer(path);
-        
+
                     if (!container.exists()) {
                         throw new IOException("Directory " + path + " does not exist.");
                     }
-                    
+
                     if (!mDfs.isContainer(path)) {
                         throw new IOException(path + " is not a directory.");
                     }
-                    
+
                     mDfs.setActiveContainer(container);
                 }
             }
             catch (DataStorageException e) {
-                throw new IOException("Failed to change working directory to " + 
-                                      ((path == null) ? (((HDataStorage)mDfs).getHFS().getHomeDirectory().toString()) 
+                throw new IOException("Failed to change working directory to " +
+                                      ((path == null) ? (((HDataStorage)mDfs).getHFS().getHomeDirectory().toString())
                                                          : (path)), e);
             }
         } else {
@@ -716,7 +722,10 @@ public class GruntParser extends PigScri
         }
 
         if(mExplain == null) { // process only if not in "explain" mode
-        	executeBatch();
+            executeBatch();
+            if ("@".equals(alias)) {
+                alias = mPigServer.getLastRel();
+            }
             Iterator<Tuple> result = mPigServer.openIterator(alias);
             while (result.hasNext())
             {
@@ -727,7 +736,7 @@ public class GruntParser extends PigScri
             log.warn("'dump' statement is ignored while processing 'explain -script' or '-check'");
         }
     }
-    
+
     @Override
     protected void processIllustrate(String alias, String script, String target, List<String> params, List<String> files) throws IOException, ParseException
     {
@@ -765,6 +774,9 @@ public class GruntParser extends PigScri
                         throw new ParseException("'illustrate' statement must be on an alias or on a script.");
                     }
                 }
+                if ("@".equals(alias)) {
+                    alias = mPigServer.getLastRel();
+                }
                 mPigServer.getExamples(alias);
             } finally {
                 if (script != null) {
@@ -784,38 +796,38 @@ public class GruntParser extends PigScri
             if (job == null)
                 System.out.println("Job with id " + jobid + " is not active");
             else
-            {    
+            {
                 job.killJob();
                 log.info("Kill " + id + " submitted.");
             }
         }
     }
-        
+
     @Override
     protected void processLS(String path) throws IOException
     {
         if(mExplain == null) { // process only if not in "explain" mode
             try {
                 ElementDescriptor pathDescriptor;
-                
+
                 if (path == null) {
                     pathDescriptor = mDfs.getActiveContainer();
                 }
                 else {
                     pathDescriptor = mDfs.asElement(path);
                 }
-    
+
                 if (!pathDescriptor.exists()) {
-                    throw new IOException("File or directory " + path + " does not exist.");                
+                    throw new IOException("File or directory " + path + " does not exist.");
                 }
-                
+
                 if (mDfs.isContainer(pathDescriptor.toString())) {
                     ContainerDescriptor container = (ContainerDescriptor) pathDescriptor;
                     Iterator<ElementDescriptor> elems = container.iterator();
-                    
+
                     while (elems.hasNext()) {
                         ElementDescriptor curElem = elems.next();
-                        
+
                         if (mDfs.isContainer(curElem.toString())) {
                                System.out.println(curElem.toString() + "\t<dir>");
                         } else {
@@ -844,9 +856,9 @@ public class GruntParser extends PigScri
 
         System.out.println(elem.toString() + "<r " + replication + ">\t" + len);
     }
-    
+
     @Override
-    protected void processPWD() throws IOException 
+    protected void processPWD() throws IOException
     {
         if(mExplain == null) { // process only if not in "explain" mode
             System.out.println(mDfs.getActiveContainer().toString());
@@ -854,19 +866,19 @@ public class GruntParser extends PigScri
             log.warn("'pwd' statement is ignored while processing 'explain -script' or '-check'");
         }
     }
-    
+
     @Override
-	protected void processHistory(boolean withNumbers) {
-    	mPigServer.printHistory(withNumbers);
+    protected void processHistory(boolean withNumbers) {
+        mPigServer.printHistory(withNumbers);
     }
-    
+
     @Override
-    protected void printHelp() 
+    protected void printHelp()
     {
         System.out.println("Commands:");
         System.out.println("<pig latin statement>; - See the PigLatin manual for details: http://hadoop.apache.org/pig");
         System.out.println("File system commands:");
-	System.out.println("    fs <fs arguments> - Equivalent to Hadoop dfs command: http://hadoop.apache.org/common/docs/current/hdfs_shell.html");	
+        System.out.println("    fs <fs arguments> - Equivalent to Hadoop dfs command: http://hadoop.apache.org/common/docs/current/hdfs_shell.html");
         System.out.println("Diagnostic commands:");
         System.out.println("    describe <alias>[::<alias] - Show the schema for the alias. Inner aliases can be described as A::B.");
         System.out.println("    explain [-script <pigscript>] [-out <path>] [-brief] [-dot] [-param <param_name>=<param_value>]");
@@ -880,17 +892,17 @@ public class GruntParser extends PigScri
         System.out.println("        alias - Alias to explain.");
         System.out.println("    dump <alias> - Compute the alias and writes the results to stdout.");
         System.out.println("Utility Commands:");
-        System.out.println("    exec [-param <param_name>=param_value] [-param_file <file_name>] <script> - ");    
+        System.out.println("    exec [-param <param_name>=param_value] [-param_file <file_name>] <script> - ");
         System.out.println("        Execute the script with access to grunt environment including aliases.");
         System.out.println("        -param <param_name - See parameter substitution for details.");
         System.out.println("        -param_file <file_name> - See parameter substitution for details.");
         System.out.println("        script - Script to be executed.");
-        System.out.println("    run [-param <param_name>=param_value] [-param_file <file_name>] <script> - ");    
+        System.out.println("    run [-param <param_name>=param_value] [-param_file <file_name>] <script> - ");
         System.out.println("        Execute the script with access to grunt environment. ");
         System.out.println("        -param <param_name - See parameter substitution for details.");
         System.out.println("        -param_file <file_name> - See parameter substitution for details.");
         System.out.println("        script - Script to be executed.");
-        System.out.println("    sh  <shell command> - Invoke a shell command."); 
+        System.out.println("    sh  <shell command> - Invoke a shell command.");
         System.out.println("    kill <job_id> - Kill the hadoop job specified by the hadoop job id.");
         System.out.println("    set <key> <value> - Provide execution parameters to Pig. Keys and values are case sensitive.");
         System.out.println("        The following keys are supported: ");
@@ -912,15 +924,15 @@ public class GruntParser extends PigScri
         if(mExplain == null) { // process only if not in "explain" mode
 
             executeBatch();
-        
+
             try {
                 ElementDescriptor srcPath = mDfs.asElement(src);
                 ElementDescriptor dstPath = mDfs.asElement(dst);
-                
+
                 if (!srcPath.exists()) {
-                    throw new IOException("File or directory " + src + " does not exist.");                
+                    throw new IOException("File or directory " + src + " does not exist.");
                 }
-                
+
                 srcPath.rename(dstPath);
             }
             catch (DataStorageException e) {
@@ -930,18 +942,18 @@ public class GruntParser extends PigScri
             log.warn("'mv' statement is ignored while processing 'explain -script' or '-check'");
         }
     }
-    
+
     @Override
     protected void processCopy(String src, String dst) throws IOException
     {
         if(mExplain == null) { // process only if not in "explain" mode
 
             executeBatch();
-        
+
             try {
                 ElementDescriptor srcPath = mDfs.asElement(src);
                 ElementDescriptor dstPath = mDfs.asElement(dst);
-                
+
                 srcPath.copy(dstPath, mConf, false);
             }
             catch (DataStorageException e) {
@@ -951,18 +963,18 @@ public class GruntParser extends PigScri
             log.warn("'cp' statement is ignored while processing 'explain -script' or '-check'");
         }
     }
-    
+
     @Override
     protected void processCopyToLocal(String src, String dst) throws IOException
     {
         if(mExplain == null) { // process only if not in "explain" mode
-            
+
             executeBatch();
-        
+
             try {
                 ElementDescriptor srcPath = mDfs.asElement(src);
                 ElementDescriptor dstPath = mLfs.asElement(dst);
-                
+
                 srcPath.copy(dstPath, false);
             }
             catch (DataStorageException e) {
@@ -977,13 +989,13 @@ public class GruntParser extends PigScri
     protected void processCopyFromLocal(String src, String dst) throws IOException
     {
         if(mExplain == null) { // process only if not in "explain" mode
-            
+
             executeBatch();
-        
+
             try {
                 ElementDescriptor srcPath = mLfs.asElement(src);
                 ElementDescriptor dstPath = mDfs.asElement(dst);
-                
+
                 srcPath.copy(dstPath, false);
             }
             catch (DataStorageException e) {
@@ -993,7 +1005,7 @@ public class GruntParser extends PigScri
             log.warn("'copyFromLocal' statement is ignored while processing 'explain -script' or '-check'");
         }
     }
-    
+
     @Override
     protected void processMkdir(String dir) throws IOException
     {
@@ -1004,7 +1016,7 @@ public class GruntParser extends PigScri
             log.warn("'mkdir' statement is ignored while processing 'explain -script' or '-check'");
         }
     }
-    
+
     @Override
     protected void processPig(String cmd) throws IOException
     {
@@ -1012,11 +1024,11 @@ public class GruntParser extends PigScri
         if (!mInteractive) {
             start = getLineNumber();
         }
-        
+
         if (cmd.charAt(cmd.length() - 1) != ';') {
             mPigServer.registerQuery(cmd + ";", start);
         }
-        else { 
+        else {
             mPigServer.registerQuery(cmd, start);
         }
     }
@@ -1026,16 +1038,16 @@ public class GruntParser extends PigScri
     {
         if(mExplain == null) { // process only if not in "explain" mode
 
-            ElementDescriptor dfsPath = mDfs.asElement(path);    
+            ElementDescriptor dfsPath = mDfs.asElement(path);
             executeBatch();
-        
+
             if (!dfsPath.exists()) {
                 if (options == null || !options.equalsIgnoreCase("force")) {
-                    throw new IOException("File or directory " + path + " does not exist."); 
+                    throw new IOException("File or directory " + path + " does not exist.");
                 }
             }
             else {
-                
+
                 dfsPath.delete();
             }
         } else {
@@ -1046,17 +1058,17 @@ public class GruntParser extends PigScri
     @Override
     protected void processFsCommand(String[] cmdTokens) throws IOException{
         if(mExplain == null) { // process only if not in "explain" mode
-            
+
             executeBatch();
-            
+
             int retCode = -1;
-            
+
             try {
                 retCode = shell.run(cmdTokens);
             } catch (Exception e) {
                 throw new IOException(e);
             }
-            
+
             if (retCode != 0 && !mInteractive) {
                 String s = LoadFunc.join(
                         (AbstractList<String>) Arrays.asList(cmdTokens), " ");
@@ -1067,13 +1079,13 @@ public class GruntParser extends PigScri
             log.warn("'fs' statement is ignored while processing 'explain -script' or '-check'");
         }
     }
-    
+
     @Override
     protected void processShCommand(String[] cmdTokens) throws IOException{
         if(mExplain == null) { // process only if not in "explain" mode
             try {
                 executeBatch();
-                
+
                 // For sh command, create a process with the following syntax
                 // <shell exe> <invoke arg> <command-as-string>
                 String  shellName = "sh";
@@ -1104,10 +1116,10 @@ public class GruntParser extends PigScri
 
                 StreamPrinter outPrinter = new StreamPrinter(executor.getInputStream(), null, System.out);
                 StreamPrinter errPrinter = new StreamPrinter(executor.getErrorStream(), null, System.err);
-    
+
                 outPrinter.start();
                 errPrinter.start();
-    
+
                 int ret = executor.waitFor();
                 outPrinter.join();
                 errPrinter.join();
@@ -1124,13 +1136,13 @@ public class GruntParser extends PigScri
             log.warn("'sh' statement is ignored while processing 'explain -script' or '-check'");
         }
     }
-    
+
     public static int runSQLCommand(String hcatBin, String cmd, boolean mInteractive) throws IOException {
         String[] tokens = new String[3];
         tokens[0] = hcatBin;
         tokens[1] = "-e";
         tokens[2] = cmd.substring(cmd.indexOf("sql")).substring(4);
-        
+
         // create new environment = environment - HADOOP_CLASSPATH
         // This is because of antlr version conflict between Pig and Hive
         Map<String, String> envs = System.getenv();
@@ -1140,7 +1152,7 @@ public class GruntParser extends PigScri
                 envSet.add(entry.getKey() + "=" + entry.getValue());
             }
         }
-        
+
         log.info("Going to run hcat command: " + tokens[2]);
         Process executor = Runtime.getRuntime().exec(tokens, envSet.toArray(new String[0]));
         StreamPrinter outPrinter = new StreamPrinter(executor.getInputStream(), null, System.out);
@@ -1148,11 +1160,11 @@ public class GruntParser extends PigScri
 
         outPrinter.start();
         errPrinter.start();
-        
+
         int ret;
         try {
             ret = executor.waitFor();
-            
+
             outPrinter.join();
             errPrinter.join();
             if (ret != 0 && !mInteractive) {
@@ -1164,7 +1176,7 @@ public class GruntParser extends PigScri
         }
         return 0;
     }
-    
+
     @Override
     protected void processSQLCommand(String cmd) throws IOException{
         if(mExplain == null) { // process only if not in "explain" mode
@@ -1184,43 +1196,43 @@ public class GruntParser extends PigScri
             log.warn("'sql' statement is ignored while processing 'explain -script' or '-check'");
         }
     }
-    
+
     /**
      * StreamPrinter.
      *
      */
     public static class StreamPrinter extends Thread {
-    	InputStream is;
-	    String type;
-	    PrintStream os;
-
-	    public StreamPrinter(InputStream is, String type, PrintStream os) {
-	    	this.is = is;
-	    	this.type = type;
-	    	this.os = os;
-	    }
-
-	    @Override
-	    public void run() {
-	        try {
-	        	InputStreamReader isr = new InputStreamReader(is);
-	        	BufferedReader br = new BufferedReader(isr);
-	        	String line = null;
-	        	if (type != null) {
-		            while ((line = br.readLine()) != null) {
-		            	os.println(type + ">" + line);
-		            }
-	        	} else {
-	        		while ((line = br.readLine()) != null) {
-	        			os.println(line);
-	        		}
-	        	}
-	        } catch (IOException ioe) {
-	        	ioe.printStackTrace();
-	        }
-	    }
+        InputStream is;
+        String type;
+        PrintStream os;
+
+        public StreamPrinter(InputStream is, String type, PrintStream os) {
+            this.is = is;
+            this.type = type;
+            this.os = os;
+        }
+
+        @Override
+        public void run() {
+            try {
+                InputStreamReader isr = new InputStreamReader(is);
+                BufferedReader br = new BufferedReader(isr);
+                String line = null;
+                if (type != null) {
+                    while ((line = br.readLine()) != null) {
+                        os.println(type + ">" + line);
+                    }
+                } else {
+                    while ((line = br.readLine()) != null) {
+                        os.println(line);
+                    }
+                }
+            } catch (IOException ioe) {
+                ioe.printStackTrace();
+            }
+        }
     }
-    
+
     private static class ExplainState {
         public long mTime;
         public int mCount;
@@ -1242,7 +1254,7 @@ public class GruntParser extends PigScri
             mFormat = format;
             mLast = false;
         }
-    }        
+    }
 
     private PigServer mPigServer;
     private DataStorage mDfs;
@@ -1256,5 +1268,5 @@ public class GruntParser extends PigScri
     private int mNumSucceededJobs;
     private FsShell shell;
     private boolean mScriptIllustrate;
-    
+
 }

Modified: pig/trunk/src/org/apache/pig/tools/pigscript/parser/PigScriptParser.jj
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/tools/pigscript/parser/PigScriptParser.jj?rev=1452909&r1=1452908&r2=1452909&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/tools/pigscript/parser/PigScriptParser.jj (original)
+++ pig/trunk/src/org/apache/pig/tools/pigscript/parser/PigScriptParser.jj Tue Mar  5 17:25:46 2013
@@ -42,13 +42,13 @@ public abstract class PigScriptParser
 {
 	protected boolean mInteractive;
 	protected ConsoleReader mConsoleReader;
-	
+
 	public void setInteractive(boolean interactive)
 	{
 		mInteractive = interactive;
 		token_source.interactive = interactive;
 	}
-	
+
     public int getLineNumber()
     {
         return jj_input_stream.getBeginLine();
@@ -95,7 +95,7 @@ public abstract class PigScriptParser
 	abstract protected void processKill(String jobid) throws IOException;
 
 	abstract protected void processLS(String path) throws IOException;
-	
+
 	abstract protected void processPWD() throws IOException;
 
 	abstract protected void printHelp();
@@ -107,13 +107,13 @@ public abstract class PigScriptParser
 	abstract protected void processCopyToLocal(String src, String dst) throws IOException;
 
 	abstract protected void processCopyFromLocal(String src, String dst) throws IOException;
-	
+
 	abstract protected void processMkdir(String dir) throws IOException;
-	
+
 	abstract protected void processPig(String cmd) throws IOException;
 
 	abstract protected void processRemove(String path, String opt) throws IOException;
-	
+
 	abstract protected void processIllustrate(String alias, String script, String target, List<String> params, List<String> files) throws IOException, ParseException;
 
 	abstract protected void processScript(String script, boolean batch, List<String> params, List<String> files) throws IOException, ParseException;
@@ -187,7 +187,7 @@ TOKEN: {<N: "-n">}
 // internal use commands
 TOKEN: {<SCRIPT_DONE: "scriptDone">}
 
-// Define pig command as 
+// Define pig command as
 // (1) Starting with "split"/"define"/"store"/"import" or assignment (A=) or multi-assignment (A, B, ...=) followed by
 // (2) Single statement followed by ; and newline or
 // (3) Block of statements enclosed in
@@ -232,7 +232,7 @@ TOKEN_MGR_DECLS : {
 {
     <";"> : SQL_END
 |   <("\n" | "\r" | "\r\n")> {secondary_prompt();}
-|   <(~[])> 
+|   <(~[])>
 }
 
 <SQL_END> TOKEN : {
@@ -247,6 +247,7 @@ TOKEN_MGR_DECLS : {
 |	<"store"> : PIG_START
 |	<"import"> : PIG_START
 | 	<(["a"-"z", "A"-"Z"])+(["a"-"z", "A"-"Z"] | ["0"-"9"] | "_")*(" " | "\t")*"="> : PIG_START
+|   <"=>" (" " | "\t")*> : PIG_START
 | 	< <IDENTIFIER> (" " | "\t")* ("," (" " | "\t")* <IDENTIFIER> )* (" " | "\t")* "="> : PIG_START
 |   < <IDENTIFIER> (" " | "\t")* "(" > : PIG_START
 }
@@ -263,7 +264,7 @@ TOKEN_MGR_DECLS : {
 |	<"--"> {prevState = PIG_START;} : SINGLE_LINE_COMMENT
 |	<"/*"> {prevState = PIG_START;} : MULTI_LINE_COMMENT
 |	<("\n" | "\r" | "\r\n")> {secondary_prompt();}
-|       <(~[])> 
+|       <(~[])>
 }
 
 <SINGLE_LINE_COMMENT> MORE :
@@ -297,20 +298,20 @@ TOKEN_MGR_DECLS : {
 
 <GENERATE> MORE :
 {
-    <"{"> 
+    <"{">
     {
-        bagConstantLevel++; 
+        bagConstantLevel++;
         prevState = getState(prevState);
         saveState(prevState);
         prevState = GENERATE;
     } : BAG_CONSTANT
-|	<(" " | "\t")+["A","a"]["S","s"](" " | "\t")+> 
+|	<(" " | "\t")+["A","a"]["S","s"](" " | "\t")+>
     {
         prevState = getState(prevState);
         saveState(prevState);
-        prevState = GENERATE; 
+        prevState = GENERATE;
      } : SCHEMA_DEFINITION
-|	<";"> 
+|	<";">
     {
         prevState = getState(prevState);
         if(prevState == PIG_START) {
@@ -320,7 +321,7 @@ TOKEN_MGR_DECLS : {
         SwitchTo(prevState);
     }
 |	<("\n" | "\r" | "\r\n")> {secondary_prompt();}
-|	<(~[])> 
+|	<(~[])>
 }
 
 <SCHEMA_DEFINITION> MORE :
@@ -329,7 +330,7 @@ TOKEN_MGR_DECLS : {
 |   <")"> {tupleSchemaLevel--; if ((tupleSchemaLevel == 0) && (bagSchemaLevel == 0)) SwitchTo(prevState); }
 |   <"{"> {bagSchemaLevel++;}
 |   <"}"> {bagSchemaLevel--; if ((tupleSchemaLevel == 0) && (bagSchemaLevel == 0)) SwitchTo(prevState); }
-|	<("," | ";" )> 
+|	<("," | ";" )>
     {
         if ((tupleSchemaLevel == 0) && (bagSchemaLevel == 0)) {
             input_stream.backup(1);
@@ -393,7 +394,7 @@ TOKEN:
 |	<#INTEGER: ( <DIGIT> )+ >
 |	<#NUMBER: <INTEGER> | <FLOAT> | <FLOAT> ( ["e","E"] ([ "-","+"])? <FLOAT> )?>
 }
-
+TOKEN: {<PREVREL: "@">}
 TOKEN: {<IDENTIFIER: (<LETTER>)+(<DIGIT> | <LETTER> | <SPECIALCHAR> | "::")*>}
 TOKEN: {<PATH: (~["(", ")", ";", "\r", " ", "\t", "\n"])+>}
 TOKEN : { <QUOTEDSTRING :  "'"
@@ -417,7 +418,7 @@ void parse() throws IOException:
 }
 
 {
-	(	
+	(
 	<EOL>
 	{prompt();}
 	|
@@ -435,7 +436,7 @@ void parse() throws IOException:
 			}
 		}
 		processFsCommand(cmdTokens.toArray(new String[cmdTokens.size()]));
-	}		
+	}
 	)+
 	|
 	<SH>
@@ -452,7 +453,7 @@ void parse() throws IOException:
 			}
 		}
 		processShCommand(cmdTokens.toArray(new String[cmdTokens.size()]));
-	}		
+	}
 	)+
 	|
 	<CAT>
@@ -479,16 +480,17 @@ void parse() throws IOException:
 	<COPYFROMLOCAL>
 	t1 = GetPath()
 	t2 = GetPath()
-	{processCopyFromLocal(t1.image, t2.image);}	
+	{processCopyFromLocal(t1.image, t2.image);}
 	|
 	<COPYTOLOCAL>
 	t1 = GetPath()
 	t2 = GetPath()
-	{processCopyToLocal(t1.image, t2.image);}	
+	{processCopyToLocal(t1.image, t2.image);}
 	|
 	(<DUMP> | <DUMP_SHORT>)
 	(
-		t1 = <IDENTIFIER>
+	        (t1 = <IDENTIFIER>
+                | t1 = <PREVREL>)
 		{processDump(t1.image);}
 		|
 		{processDump(null);}
@@ -498,7 +500,8 @@ void parse() throws IOException:
 	|
 	(<DESCRIBE> | <DESCRIBE_SHORT>)
 	(
-		t1 = <IDENTIFIER>
+		(t1 = <IDENTIFIER>
+                | t1 = <PREVREL>)
 		{processDescribe(t1.image);}
 		|
 		{processDescribe(null);}
@@ -515,7 +518,7 @@ void parse() throws IOException:
 	History()
 	|
 	<KILL>
-	t1 = <IDENTIFIER>	
+	t1 = <IDENTIFIER>
 	{processKill(t1.image);}
 	|
 	<LS>
@@ -556,7 +559,7 @@ void parse() throws IOException:
 			<AS> t3 = <IDENTIFIER>
 			{namespace = t3.image; }
 		]
-	]	
+	]
 	{processRegister(unquote(t1.image), engine, namespace); }
 	|
 	Script()
@@ -616,7 +619,7 @@ void Illustrate() throws IOException:
 {
 	(<ILLUSTRATE> | <ILLUSTRATE_SHORT>)
 	{
-		params = new ArrayList<String>(); 
+		params = new ArrayList<String>();
 		files = new ArrayList<String>();
 	}
 	(
@@ -637,7 +640,8 @@ void Illustrate() throws IOException:
 		{files.add(t.image);}
 	)*
 	(
-		t = <IDENTIFIER>
+		(t = <IDENTIFIER>
+                | t = <PREVREL>)
 		{alias = t.image;}
 	)?
 	{processIllustrate(alias, script, target, params, files);}
@@ -671,7 +675,7 @@ void Explain() throws IOException:
 {
 	(<EXPLAIN> | <EXPLAIN_SHORT>)
 	{
-		params = new ArrayList<String>(); 
+		params = new ArrayList<String>();
 		files = new ArrayList<String>();
 	}
 	(
@@ -698,7 +702,8 @@ void Explain() throws IOException:
 		{files.add(t.image);}
 	)*
 	(
-		t = <IDENTIFIER>
+		(t = <IDENTIFIER>
+                | t = <PREVREL>)
 		{alias = t.image;}
 	)?
 	{processExplain(alias, script, isVerbose, format, target, params, files);}
@@ -721,7 +726,7 @@ void Script() throws IOException:
 		{batch = true;}
 	)
     	{
-		params = new ArrayList<String>(); 
+		params = new ArrayList<String>();
 		files = new ArrayList<String>();
 	}
 	(
@@ -756,7 +761,7 @@ Token GetPath() :
 }
 
 Token GetKey() :
-{	
+{
 	Token t;
 }
 {
@@ -852,8 +857,8 @@ Token GetReserved () :
 	{return t;}
 }
 
-JAVACODE 
-void handle_invalid_command(int kind) 
+JAVACODE
+void handle_invalid_command(int kind)
 {
 	throw generateParseException();  // throw the exception object.
 }

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=1452909&r1=1452908&r2=1452909&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestBuiltin.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestBuiltin.java Tue Mar  5 17:25:46 2013
@@ -1011,9 +1011,9 @@ public class TestBuiltin {
     public void testCount_ValidNumberOfArguments_WithoutInputSchema_One() throws Exception {
          File inputFile = createCountInputFile();
          try {
-             pigServer.registerQuery("A = load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "';");
-             pigServer.registerQuery("B = group A all;");
-             pigServer.registerQuery("C = foreach B generate COUNT(A.$0);");
+             pigServer.registerQuery("=> load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "';");
+             pigServer.registerQuery("=> group @ all;");
+             pigServer.registerQuery("C = foreach @ generate COUNT($1.$0);");
              assertValidCount();
          }catch(TypeCheckerException e) {
              Assert.fail("Query is in accordance with schema, still it failed to execute");
@@ -1026,9 +1026,9 @@ public class TestBuiltin {
     public void testCount_ValidNumberOfArguments_WithoutInputSchema_Two() throws Exception {
          File inputFile = createCountInputFile();
          try {
-             pigServer.registerQuery("A = load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "';");
-             pigServer.registerQuery("B = group A all;");
-             pigServer.registerQuery("C = foreach B generate COUNT(A);");
+             pigServer.registerQuery("=> load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "';");
+             pigServer.registerQuery("=> group @ all;");
+             pigServer.registerQuery("C = foreach @ generate COUNT($1);");
              assertValidCount();
          }catch(TypeCheckerException e) {
              Assert.fail("Query is in accordance with schema, still it failed to execute");
@@ -1049,9 +1049,9 @@ public class TestBuiltin {
         File inputFile = Util.createInputFile("tmp", inputFileName, inputData);
 
          try {
-             pigServer.registerQuery("A = load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "';");
-             pigServer.registerQuery("B = group A all;");
-             pigServer.registerQuery("C = foreach B generate COUNT(A);");
+             pigServer.registerQuery("=> load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "';");
+             pigServer.registerQuery("=> group @ all;");
+             pigServer.registerQuery("C = foreach @ generate COUNT($1);");
              assertValidCount();
          }catch(TypeCheckerException e) {
              Assert.fail("Query is in accordance with schema, still it failed to execute");
@@ -1072,9 +1072,9 @@ public class TestBuiltin {
         String inputFileName = file.getAbsolutePath();
 
          try {
-             pigServer.registerQuery("A = load '" + Util.encodeEscape(inputFileName) + "';");
-             pigServer.registerQuery("B = group A all;");
-             pigServer.registerQuery("C = foreach B generate COUNT(A.$0);");
+             pigServer.registerQuery("=> load '" + Util.encodeEscape(inputFileName) + "';");
+             pigServer.registerQuery("=> group @ all;");
+             pigServer.registerQuery("C = foreach @ generate COUNT($1.$0);");
              assertValidCount();
          }catch(TypeCheckerException e) {
              Assert.fail("Query is in accordance with schema, still it failed to execute");
@@ -1088,9 +1088,9 @@ public class TestBuiltin {
     public void testCount_ValidNumberOfArguments_WithInputSchema_One() throws Exception {
          File inputFile = createCountInputFile();
          try {
-             pigServer.registerQuery("A = load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "' as (data:chararray);");
-             pigServer.registerQuery("B = group A all;");
-             pigServer.registerQuery("C = foreach B generate COUNT(A.$0);");
+             pigServer.registerQuery("=> load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "' as (data:chararray);");
+             pigServer.registerQuery("=> group @ all;");
+             pigServer.registerQuery("C = foreach @ generate COUNT($1.$0);");
              assertValidCount();
          }catch(TypeCheckerException e) {
              Assert.fail("Query is in accordance with schema, still it failed to execute");
@@ -1103,9 +1103,9 @@ public class TestBuiltin {
     public void testCount_ValidNumberOfArguments_WithInputSchema_Two() throws Exception {
          File inputFile = createCountInputFile();
          try {
-             pigServer.registerQuery("A = load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "' as (data:chararray);");
-             pigServer.registerQuery("B = group A all;");
-             pigServer.registerQuery("C = foreach B generate COUNT(A);");
+             pigServer.registerQuery("=> load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "' as (data:chararray);");
+             pigServer.registerQuery("=> group @ all;");
+             pigServer.registerQuery("C = foreach @ generate COUNT($1);");
              assertValidCount();
          }catch(TypeCheckerException e) {
              Assert.fail("Query is in accordance with schema, still it failed to execute");
@@ -1139,9 +1139,9 @@ public class TestBuiltin {
     public void testCount_InvalidNumberOfArguments_WithoutInputSchema() throws Exception {
          File inputFile = createCountInputFile();
          try {
-             pigServer.registerQuery("A = load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "';");
-             pigServer.registerQuery("B = group A all;");
-             pigServer.registerQuery("C = foreach B generate COUNT(A.$0, A.$0);");
+             pigServer.registerQuery("=> load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "';");
+             pigServer.registerQuery("=> group @ all;");
+             pigServer.registerQuery("C = foreach @ generate COUNT($1.$0, $1.$0);");
              pigServer.openIterator("C");
              Assert.fail("COUNT is suppose to run with one argument of type BAG, however it ran with couple of arguments.");
          }catch(FrontendException e) {
@@ -1155,9 +1155,9 @@ public class TestBuiltin {
     public void testCount_InvalidNumberOfArguments_WithInputSchema() throws Exception {
          File inputFile = createCountInputFile();
          try {
-             pigServer.registerQuery("A = load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "' as (data:chararray);");
-             pigServer.registerQuery("B = group A all;");
-             pigServer.registerQuery("C = foreach B generate COUNT(A.$0, A.$0);");
+             pigServer.registerQuery("=> load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "' as (data:chararray);");
+             pigServer.registerQuery("=> group @ all;");
+             pigServer.registerQuery("C = foreach @ generate COUNT($1.$0, $1.$0);");
              pigServer.openIterator("C");
              Assert.fail("COUNT is suppose to run with one argument of type BAG, however it ran with couple of arguments.");
          }catch(FrontendException e) {
@@ -1171,9 +1171,9 @@ public class TestBuiltin {
     public void testCount_InvalidArgumentType() throws Exception {
          File inputFile = createCountInputFile();
          try {
-             pigServer.registerQuery("A = load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "' as (data:chararray);");
-             pigServer.registerQuery("B = group A all;");
-             pigServer.registerQuery("C = foreach B generate COUNT('data');");
+             pigServer.registerQuery("=> load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "' as (data:chararray);");
+             pigServer.registerQuery("=> group @ all;");
+             pigServer.registerQuery("C = foreach @ generate COUNT('data');");
              pigServer.openIterator("C");
              Assert.fail("COUNT is suppose to run with one argument of type BAG, however it ran with an argument of type chararray.");
          }catch(FrontendException e) {
@@ -2223,9 +2223,9 @@ public class TestBuiltin {
         File inputFile = Util.createInputFile("tmp", inputFileName, inputData);
 
         // test typed data
-        pigServer.registerQuery("A = load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "' AS value:chararray;");
-        pigServer.registerQuery("B = foreach A generate STRSPLIT(value, ' ') AS values;");
-        pigServer.registerQuery("C = foreach B generate values, SIZE(values) as cnt;");
+        pigServer.registerQuery("=> load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "' AS value:chararray;");
+        pigServer.registerQuery("=> foreach @ generate STRSPLIT(value, ' ') AS values;");
+        pigServer.registerQuery("C = foreach @ generate values, SIZE(values) as cnt;");
 
         Iterator<Tuple> it = pigServer.openIterator("C");
         int i=0;
@@ -2370,8 +2370,8 @@ public class TestBuiltin {
         File inputFile = Util.createInputFile("tmp", "testStrUDFsIn.txt", new String[] {inputStr});
 
         // test typed data
-        pigServer.registerQuery("A = load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "' as (name: chararray);");
-        pigServer.registerQuery("B = foreach A generate SUBSTRING(name, 0, 3), " +
+        pigServer.registerQuery("=> load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "' as (name: chararray);");
+        pigServer.registerQuery("B = foreach @ generate SUBSTRING(name, 0, 3), " +
             "INDEXOF(name, 'a'), INDEXOF(name, 'a', 3), LAST_INDEX_OF(name, 'a'), REPLACE(name, 'a', 'b'), " +
             "STRSPLIT(name), STRSPLIT(name, ' '), STRSPLIT(name, ' ', 0), TRIM(name);");
 
@@ -2392,8 +2392,8 @@ public class TestBuiltin {
         assertEquals("amy smith", t.get(8));
 
         // test untyped data
-        pigServer.registerQuery("A = load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "' as (name);");
-        pigServer.registerQuery("B = foreach A generate SUBSTRING(name, 0, 3), " +
+        pigServer.registerQuery("=> load '" + Util.encodeEscape(inputFile.getAbsolutePath()) + "' as (name);");
+        pigServer.registerQuery("B = foreach @ generate SUBSTRING(name, 0, 3), " +
             "LAST_INDEX_OF(name, 'a'), REPLACE(name, 'a', 'b'), TRIM(name);");
 
         it = pigServer.openIterator("B");

Modified: pig/trunk/test/org/apache/pig/test/TestShortcuts.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestShortcuts.java?rev=1452909&r1=1452908&r2=1452909&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestShortcuts.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestShortcuts.java Tue Mar  5 17:25:46 2013
@@ -2,32 +2,43 @@ package org.apache.pig.test;
 
 import static org.apache.pig.builtin.mock.Storage.resetData;
 import static org.apache.pig.builtin.mock.Storage.tuple;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.BufferedReader;
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
+import java.util.Iterator;
+import java.util.Set;
 
+import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.builtin.mock.Storage.Data;
+import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.impl.util.Utils;
 import org.apache.pig.tools.grunt.Grunt;
 import org.apache.pig.tools.pigscript.parser.ParseException;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.collect.Sets;
+
 public class TestShortcuts {
     private String basedir = "test/org/apache/pig/test/data";
     private Data data;
-    private PigServer server;
+    private PigServer pigServer;
     private PigContext context;
 
     @Before
     public void setup() throws ExecException, IOException {
-        server = new PigServer("local");
-        context = server.getPigContext();
-        data = resetData(server);
+        pigServer = new PigServer(ExecType.LOCAL);
+        context = pigServer.getPigContext();
+        data = resetData(pigServer);
         data.set("input", tuple("dog", "miami", 12), tuple("cat", "miami", 18), tuple("turtle", "tampa", 4),
                 tuple("dog", "tampa", 14), tuple("cat", "naples", 9), tuple("dog", "naples", 5),
                 tuple("turtle", "naples", 1));
@@ -246,4 +257,55 @@ public class TestShortcuts {
         Grunt grunt = new Grunt(new BufferedReader(reader), context);
         grunt.exec();
     }
+
+    @Test
+    public void testDumpWithPreviousRelation() throws Exception {
+        Data data = resetData(pigServer);
+        Set<Tuple> expected = Sets.newHashSet(tuple("a"), tuple("b"), tuple("c"));
+
+        data.set("foo", Utils.getSchemaFromString("x:chararray"), expected);
+        pigServer.registerQuery("=> load 'foo' using mock.Storage();");
+        Iterator<Tuple> op = pigServer.openIterator("@");
+        while (op.hasNext()) {
+            assertTrue(expected.remove(op.next()));
+        }
+        assertFalse(op.hasNext());
+        assertTrue(expected.isEmpty());
+    }
+    
+    @Test
+    public void testDescribeWithPreviousRelation() throws Exception {
+        Data data = resetData(pigServer);
+        Set<Tuple> expected = Sets.newHashSet(tuple("a"), tuple("b"), tuple("c"));
+
+        Schema s = Utils.getSchemaFromString("x:chararray");
+        data.set("foo", s, expected);
+        pigServer.registerQuery("=> load 'foo' using mock.Storage();");
+        Schema s2 = pigServer.dumpSchema("@");
+        assertEquals(s,s2);
+    }
+    
+    @Test
+    public void testExplainWithPreviousRelation() throws Throwable {
+        String cmd = "=> load 'input' USING mock.Storage() as (x:chararray,y:chararray,z:long);"
+                + "explain @;";
+
+        ByteArrayInputStream cmdstream = new ByteArrayInputStream(cmd.getBytes());
+        InputStreamReader reader = new InputStreamReader(cmdstream);
+
+        Grunt grunt = new Grunt(new BufferedReader(reader), context);
+        grunt.exec();
+    }
+    
+    @Test
+    public void testIllustrateWithPreviousRelation() throws Throwable {
+        String cmd = "=> load 'input' USING mock.Storage() as (x:chararray,y:chararray,z:long);"
+                + "illustrate @;";
+
+        ByteArrayInputStream cmdstream = new ByteArrayInputStream(cmd.getBytes());
+        InputStreamReader reader = new InputStreamReader(cmdstream);
+
+        Grunt grunt = new Grunt(new BufferedReader(reader), context);
+        grunt.exec();
+    }
 }