You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ol...@apache.org on 2007/12/06 19:32:22 UTC

svn commit: r601805 - in /incubator/pig/trunk: ./ src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/tools/grunt/ src/org/apache/pig/tools/pigscript/ src/org/apache/pig/tools/pigscript/parser/

Author: olston
Date: Thu Dec  6 10:32:19 2007
New Revision: 601805

URL: http://svn.apache.org/viewvc?rev=601805&view=rev
Log:
abstract Pig Script parser (jira issue #PIG-38)

Added:
    incubator/pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java
    incubator/pig/trunk/src/org/apache/pig/tools/pigscript/
    incubator/pig/trunk/src/org/apache/pig/tools/pigscript/parser/
    incubator/pig/trunk/src/org/apache/pig/tools/pigscript/parser/PigScriptParser.jj
Removed:
    incubator/pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.jj
Modified:
    incubator/pig/trunk/build.xml
    incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOCogroup.java
    incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOEval.java

Modified: incubator/pig/trunk/build.xml
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/build.xml?rev=601805&r1=601804&r2=601805&view=diff
==============================================================================
--- incubator/pig/trunk/build.xml (original)
+++ incubator/pig/trunk/build.xml Thu Dec  6 10:32:19 2007
@@ -41,13 +41,13 @@
 		<delete file="${src.dir}/org/apache/pig/impl/logicalLayer/parser/SimpleNode.java" />
 		<delete file="${src.dir}/org/apache/pig/impl/logicalLayer/parser/Token.java" />
 		<delete file="${src.dir}/org/apache/pig/impl/logicalLayer/parser/TokenMgrError.java" />
-		<delete file="${src.dir}/org/apache/pig/tools/grunt/GruntParser.java" />
-		<delete file="${src.dir}/org/apache/pig/tools/grunt/GruntParserConstants.java" />
-		<delete file="${src.dir}/org/apache/pig/tools/grunt/GruntParserTokenManager.java" />
-		<delete file="${src.dir}/org/apache/pig/tools/grunt/ParseException.java" />
-		<delete file="${src.dir}/org/apache/pig/tools/grunt/SimpleCharStream.java" />
-		<delete file="${src.dir}/org/apache/pig/tools/grunt/Token.java" />
-		<delete file="${src.dir}/org/apache/pig/tools/grunt/TokenMgrError.java" />
+		<delete file="${src.dir}/org/apache/pig/tools/pigscript/parser/PigScriptParser.java" />
+		<delete file="${src.dir}/org/apache/pig/tools/pigscript/parser/PigScriptParserConstants.java" />
+		<delete file="${src.dir}/org/apache/pig/tools/pigscript/parser/PigScriptParserTokenManager.java" />
+		<delete file="${src.dir}/org/apache/pig/tools/pigscript/parser/ParseException.java" />
+		<delete file="${src.dir}/org/apache/pig/tools/pigscript/parser/SimpleCharStream.java" />
+		<delete file="${src.dir}/org/apache/pig/tools/pigscript/parser/Token.java" />
+		<delete file="${src.dir}/org/apache/pig/tools/pigscript/parser/TokenMgrError.java" />
 	</target>
 
     <target name="depend">
@@ -80,8 +80,8 @@
 			outputdirectory="${src.dir}/org/apache/pig/impl/logicalLayer/parser"
 			javacchome="${basedir}/lib" />
 		<javacc
-			target="${src.dir}/org/apache/pig/tools/grunt/GruntParser.jj" 
-			outputdirectory="${src.dir}/org/apache/pig/tools/grunt"
+			target="${src.dir}/org/apache/pig/tools/pigscript/parser/PigScriptParser.jj" 
+			outputdirectory="${src.dir}/org/apache/pig/tools/pigscript/parser"
 			javacchome="${basedir}/lib" />
 	</target>
 

Modified: incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOCogroup.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOCogroup.java?rev=601805&r1=601804&r2=601805&view=diff
==============================================================================
--- incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOCogroup.java (original)
+++ incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOCogroup.java Thu Dec  6 10:32:19 2007
@@ -44,7 +44,8 @@
 
     @Override
     public String name() {
-        return "CoGroup";
+    	if (inputs.size() == 1) return "Group";
+    	else return "CoGroup";
     }
     @Override
     public String arguments() {

Modified: incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOEval.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOEval.java?rev=601805&r1=601804&r2=601805&view=diff
==============================================================================
--- incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOEval.java (original)
+++ incubator/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOEval.java Thu Dec  6 10:32:19 2007
@@ -37,7 +37,7 @@
 
     @Override
     public String name() {
-        return "Eval";
+        return "Foreach";
     }
 
     @Override

Added: incubator/pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java?rev=601805&view=auto
==============================================================================
--- incubator/pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java (added)
+++ incubator/pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java Thu Dec  6 10:32:19 2007
@@ -0,0 +1,303 @@
+package org.apache.pig.tools.grunt;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Reader;
+import java.util.Iterator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.pig.PigServer;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.tools.pigscript.parser.ParseException;
+import org.apache.pig.tools.pigscript.parser.PigScriptParser;
+import org.apache.pig.tools.pigscript.parser.PigScriptParserTokenManager;
+
+public class GruntParser extends PigScriptParser {
+
+
+    public GruntParser(Reader stream) {
+		super(stream);
+		init();
+    }
+
+	public GruntParser(InputStream stream, String encoding) {
+		super(stream, encoding);
+		init();
+	}
+
+	public GruntParser(InputStream stream) {
+		super(stream);
+		init();
+	}
+
+	public GruntParser(PigScriptParserTokenManager tm) {
+		super(tm);
+		init();
+	}
+
+	private void init() {
+		// nothing, for now.
+	}
+	
+	public void parseStopOnError() throws IOException, ParseException
+	{
+		prompt();
+		mDone = false;
+		while(!mDone)
+			parse();
+	}
+
+	public void parseContOnError()
+	{
+		prompt();
+		mDone = false;
+		while(!mDone)
+			try
+			{
+				parse();
+			}
+			catch(Exception e)
+			{
+				System.err.println(e.getMessage());
+			}
+	}
+
+	public void setParams(PigServer pigServer)
+	{
+		mPigServer = pigServer;
+		mDfs = mPigServer.getPigContext().getDfs();
+		mConf = mPigServer.getPigContext().getConf();
+		mJobClient = mPigServer.getPigContext().getJobClient();
+	}
+
+	public void prompt()
+	{
+		if (mInteractive)
+		{
+			System.err.print("grunt> ");
+			System.err.flush();
+		}
+	}
+	
+	protected void quit()
+	{
+		mDone = true;
+	}
+	
+	protected void processRegisterFunc(String name, String expr) {
+		mPigServer.registerFunction(name, expr);
+	}
+	
+	protected void processDescribe(String alias) throws IOException {
+		mPigServer.dumpSchema(alias);
+	}
+	
+	protected void processRegister(String jar) throws IOException {
+		mPigServer.registerJar(jar);
+	}
+
+	protected void processSet(String key, String value) throws IOException, ParseException {
+		if (key.equals("debug"))
+		{
+			if (value.equals("on") || value.equals("'on'"))
+				mPigServer.debugOn();
+			else if (value.equals("off") || value.equals("'off'"))
+				mPigServer.debugOff();
+			else
+				throw new ParseException("Invalid value " + value + " provided for " + key);
+		}
+		else if (key.equals("job.name"))
+		{
+			//mPigServer.setJobName(unquote(value));
+			mPigServer.setJobName(value);
+		}
+		else
+		{
+			// other key-value pairs can go there
+			// for now just throw exception since we don't support
+			// anything else
+			throw new ParseException("Unrecognized set key: " + key);
+		}
+	}
+	
+	protected void processStore(String alias, String file, String func) throws IOException {
+		mPigServer.store(alias, file, func);
+	}
+		
+	protected void processCat(String path) throws IOException
+	{
+		byte buffer[] = new byte[65536];
+		Path dfsPath = new Path(path);
+		int rc;
+		
+		if (!mDfs.exists(dfsPath))
+			throw new IOException("Directory " + path + " does not exist.");
+
+		if (mDfs.getFileStatus(dfsPath).isDir()) 
+		{
+			FileStatus fileStat[] = mDfs.listStatus(dfsPath);
+			for (int j = 0; j < fileStat.length; j++)
+			{
+				Path curPath = fileStat[j].getPath();
+				if (!mDfs.isFile(curPath)) continue;
+				FSDataInputStream is = mDfs.open(curPath);
+				while ((rc = is.read(buffer)) > 0)
+					System.out.write(buffer, 0, rc);
+				is.close();
+			}
+		} 
+		else 
+		{
+			FSDataInputStream is = mDfs.open(dfsPath);
+			while ((rc = is.read(buffer)) > 0)
+				System.out.write(buffer, 0, rc);
+			is.close();
+		}
+	}
+
+	protected void processCD(String path) throws IOException
+	{	
+		if (path == null)
+			mDfs.setWorkingDirectory(new Path("/user/" + System.getProperty("user.name")));
+		else
+		{
+			Path dfsDir = new Path(path);
+
+			if (!mDfs.exists(dfsDir))
+				throw new IOException("Directory " + path + " does not exist.");
+
+			if (!mDfs.getFileStatus(dfsDir).isDir())
+				throw new IOException(path + " is not a directory.");
+
+			mDfs.setWorkingDirectory(dfsDir);
+		}
+	}
+
+	protected void processDump(String alias) throws IOException
+	{
+		Iterator result = mPigServer.openIterator(alias);
+		while (result.hasNext())
+		{
+			Tuple t = (Tuple) result.next();
+			System.out.println(t);
+		}
+	}
+
+	protected void processKill(String jobid) throws IOException
+	{
+		RunningJob job = mJobClient.getJob(jobid);
+		if (job == null)
+			System.out.println("Job with id " + jobid + " is not active");
+		else
+		{	
+			job.killJob();
+			System.err.println("kill submited.");
+		}
+	}
+
+	protected void processLS(String path) throws IOException
+	{
+		Path dir;
+		if(path == null)
+                	dir = mDfs.getWorkingDirectory();
+		else
+		{
+			dir = new Path(path);
+			if (!mDfs.exists(dir))
+				throw new IOException("File or directory " + path + " does not exist.");
+		}
+
+		FileStatus fileStat[] = mDfs.listStatus(dir);
+		for (int j = 0; j < fileStat.length; j++)
+		{
+            if (fileStat[j].isDir())
+           		System.out.println(fileStat[j].getPath() + "\t<dir>");
+			else
+				System.out.println(fileStat[j].getPath() + "<r " + fileStat[j].getReplication() + ">\t" + fileStat[j].getLen());
+                }
+	}
+	
+	protected void processPWD() throws IOException 
+	{
+		System.out.println(mDfs.getWorkingDirectory());
+	}
+
+	protected void printHelp() 
+	{
+		System.err.println("Commands:");
+		System.err.println("<pig latin statement>;");
+		System.err.println("store <alias> into <filename> [using <functionSpec>]");
+		System.err.println("dump <alias>");
+		System.err.println("describe <alias>");
+		System.err.println("kill <job_id>");
+		System.err.println("ls <path>\r\ndu <path>\r\nmv <src> <dst>\r\ncp <src> <dst>\r\nrm <src>");
+		System.err.println("copyFromLocal <localsrc> <dst>\r\ncd <dir>\r\npwd");
+		System.err.println("cat <src>\r\ncopyToLocal <src> <localdst>\r\nmkdir <path>");
+		System.err.println("cd <path>");
+		System.err.println("define <functionAlias> <functionSpec>");
+		System.err.println("register <udfJar>");
+		System.err.println("set key value");
+		System.err.println("quit");
+	}
+
+	protected void processMove(String src, String dst) throws IOException
+	{
+		Path srcPath = new Path(src);
+		Path dstPath = new Path(dst);
+		if (!mDfs.exists(srcPath))
+			throw new IOException("File or directory " + src + " does not exist.");
+
+		{mDfs.rename(srcPath, dstPath);}
+	}
+	
+	protected void processCopy(String src, String dst) throws IOException
+	{
+        FileUtil.copy(mDfs, new Path(src), mDfs, new Path(dst), false, mConf);
+	}
+	
+	protected void processCopyToLocal(String src, String dst) throws IOException
+	{
+		mDfs.copyToLocalFile(new Path(src), new Path(dst));
+	}
+
+	protected void processCopyFromLocal(String src, String dst) throws IOException
+	{
+        mDfs.copyFromLocalFile(new Path(src), new Path(dst));
+	}
+	
+	protected void processMkdir(String dir) throws IOException
+	{
+		mDfs.mkdirs(new Path(dir));
+	}
+	
+	protected void processPig(String cmd) throws IOException
+	{
+		if (cmd.charAt(cmd.length() - 1) != ';') 
+			mPigServer.registerQuery(cmd + ";"); 
+		else 
+			mPigServer.registerQuery(cmd);
+	}
+
+	protected void processRemove(String path) throws IOException
+	{
+		Path dfsPath = new Path(path);
+		if (!mDfs.exists(dfsPath))
+			throw new IOException("File or directory " + path + " does not exist.");
+
+		mDfs.delete(dfsPath);
+	}
+
+	private PigServer mPigServer;
+	private FileSystem mDfs;
+	private Configuration mConf;
+	private JobClient mJobClient;
+	private boolean mDone;
+
+}
\ No newline at end of file

Added: incubator/pig/trunk/src/org/apache/pig/tools/pigscript/parser/PigScriptParser.jj
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/src/org/apache/pig/tools/pigscript/parser/PigScriptParser.jj?rev=601805&view=auto
==============================================================================
--- incubator/pig/trunk/src/org/apache/pig/tools/pigscript/parser/PigScriptParser.jj (added)
+++ incubator/pig/trunk/src/org/apache/pig/tools/pigscript/parser/PigScriptParser.jj Thu Dec  6 10:32:19 2007
@@ -0,0 +1,492 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// Grammar for PigScript
+
+options {
+  // Generate non-static functions
+  STATIC = false;
+  // Case is ignored in keywords
+  IGNORE_CASE = true;
+}
+
+PARSER_BEGIN(PigScriptParser)
+package org.apache.pig.tools.pigscript.parser;
+
+import java.io.IOException;
+
+public abstract class PigScriptParser
+{
+	protected boolean mInteractive;
+
+	public void setInteractive(boolean interactive)
+	{
+		mInteractive = interactive;
+		token_source.interactive = interactive;
+	}
+	
+	abstract public void prompt();
+	
+	abstract protected void quit();
+	
+	abstract protected void processRegisterFunc(String name, String expr);
+	
+	abstract protected void processDescribe(String alias) throws IOException;
+	
+	abstract protected void processRegister(String jar) throws IOException;
+
+	abstract protected void processSet(String key, String value) throws IOException, ParseException;
+	
+	abstract protected void processStore(String alias, String file, String func) throws IOException;
+		
+	abstract protected void processCat(String path) throws IOException;
+
+	abstract protected void processCD(String path) throws IOException;
+
+	abstract protected void processDump(String alias) throws IOException;
+
+	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();
+
+	abstract protected void processMove(String src, String dst) throws IOException;
+
+	abstract protected void processCopy(String src, String dst) throws IOException;
+
+	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) throws IOException;
+
+	 static String unquote(String s)
+	{
+		if (s.charAt(0) == '\'' && s.charAt(s.length()-1) == '\'')
+			return s.substring(1, s.length()-1);
+		else
+			return s;
+	}
+
+}
+PARSER_END(PigScriptParser)
+
+// Skip all tabs and spaces
+SKIP : { " " |  "\t" }
+
+// Skip comments(single line and multiline)
+SKIP : {
+   <"--"(~["\r","\n"])*>
+|  <"/*" (~["*"])* "*" ("*" | (~["*","/"] (~["*"])* "*"))* "/">
+}
+
+// tokens
+
+// commands
+TOKEN: {<CAT: "cat">}
+TOKEN: {<CD: "cd">}
+TOKEN: {<COPY: "cp">}
+TOKEN: {<COPYFROMLOCAL: "copyFromLocal">}
+TOKEN: {<COPYTOLOCAL: "copyToLocal">}
+TOKEN: {<DEFINE: "define">}
+TOKEN: {<DUMP: "dump">}
+TOKEN: {<DESCRIBE: "describe">}
+TOKEN: {<HELP: "help">}
+TOKEN: {<KILL: "kill">}
+TOKEN: {<LS: "ls">}
+TOKEN: {<MOVE: "mv">}
+TOKEN: {<MKDIR: "mkdir">}
+TOKEN: {<PWD: "pwd">}
+TOKEN: {<QUIT: "quit">}
+TOKEN: {<REGISTER: "register">}
+TOKEN: {<REMOVE: "rm">}
+TOKEN: {<SET: "set">}
+TOKEN: {<STORE: "store">}
+TOKEN: {<INTO: "into">}
+TOKEN: {<USING: "using">}
+
+// internal use commands
+TOKEN: {<SCRIPT_DONE: "scriptDone">}
+
+// Define pig command as 
+// (1) Starting with "split" or assignment (A=) followed by
+// (2) Single statement followed by ; and newline or
+// (3) Block of statements enclosed in
+
+TOKEN_MGR_DECLS : {
+        int pigBlockLevel = 0;
+        int funcBlockLevel = 0;
+	int prevState = DEFAULT;
+	boolean interactive = false;
+	public void secondary_prompt()
+	{
+		if (interactive)
+		{
+			System.err.print(">> ");
+			System.err.flush();
+		}
+	}
+
+}
+
+<DEFAULT> MORE :
+{
+	<"split"> : PIG_START
+| 	<"("> : FUNC_ARGS_START
+| 	<(["a"-"z", "A"-"Z"])+(["a"-"z", "A"-"Z"] | ["0"-"9"] | "_")*(" " | "\t")*"="> : PIG_START
+}
+
+<PIG_START> MORE :
+{
+	<"'"> {prevState = PIG_START;} : IN_STRING
+|       <"{"> {pigBlockLevel = 1;} : IN_BLOCK
+|       <"}"> {if (true) throw new TokenMgrError("Unmatched '}'", TokenMgrError.LEXICAL_ERROR);}
+|       <";"> : PIG_END
+|	<"--"> {prevState = PIG_START;} : SINGLE_LINE_COMMENT
+|	<"/*"> {prevState = PIG_START;} : MULTI_LINE_COMMENT
+|	<("\n" | "\r" | "\r\n")> {secondary_prompt();}
+|       <(~[])>
+}
+
+<SINGLE_LINE_COMMENT> MORE :
+{
+	<("\n" | "\r" | "\r\n")> {SwitchTo(prevState); if(prevState != DEFAULT) secondary_prompt();}
+|       <(~[])>
+}
+
+<MULTI_LINE_COMMENT> MORE :
+{
+	<"*/"> {SwitchTo(prevState);}
+|	<("\n" | "\r" | "\r\n")> {secondary_prompt();}
+|       <(~[])>
+}
+
+<IN_STRING> MORE :
+{
+	<"'"> { SwitchTo(prevState);}
+|	<("\n" | "\r" | "\r\n")> {secondary_prompt();}
+|	<(~[])>
+}
+
+<IN_BLOCK> MORE :
+{
+	<"\""> {prevState = IN_BLOCK;} : IN_STRING
+|	<"{"> {pigBlockLevel++;}
+|       <"}"(";")?> {pigBlockLevel--; if (pigBlockLevel == 0) SwitchTo(PIG_END);}
+|	<"'"> {prevState = IN_BLOCK;} : IN_STRING
+|	<"--"> {prevState = IN_BLOCK;} : SINGLE_LINE_COMMENT
+|	<"/*"> {prevState = IN_BLOCK;} : MULTI_LINE_COMMENT
+|	<("\n" | "\r" | "\r\n")> {secondary_prompt();}
+|       <(~[])>
+}
+
+<PIG_END> TOKEN :
+{
+        <PIG: ""> { 	matchedToken.image = image.toString();
+			//System.out.println("image = " +  matchedToken.image);
+		}: DEFAULT
+}
+
+<FUNC_ARGS_START> MORE :
+{
+	<"("> {funcBlockLevel = 1;} : IN_ARG_BLOCK
+|	<")"> : FUNC_ARGS_END
+|	<(~[])>
+}
+
+<IN_ARG_BLOCK> MORE:
+{
+        <"("> {funcBlockLevel++;}
+|       <")"> {funcBlockLevel--; if (funcBlockLevel == 0) SwitchTo(FUNC_ARGS_END);}
+|       <(~[])>
+}
+
+<FUNC_ARGS_END> TOKEN :
+{
+        <FUNC_ARGS: ""> {matchedToken.image = image.toString();}: DEFAULT
+}
+
+// other
+TOKEN: {<EOL:  "\r" | "\n" | "\r\n">}
+TOKEN: {<QUOTE: "'">}
+TOKEN: {<SEMICOLON: ";">}
+TOKEN:
+{
+	<#LETTER : ["a"-"z", "A"-"Z"] >
+|	<#DIGIT : ["0"-"9"] >
+|	<#SPECIALCHAR : ["_"] >
+|	<#FSSPECIALCHAR: ["/"]>
+|       <#FLOAT: <INTEGER> ( "." <INTEGER> )? | "." <INTEGER> >
+|       <#INTEGER: ( <DIGIT> )+ >
+|	<#NUMBER: <INTEGER> | <FLOAT> | <FLOAT> ( ["e","E"] ([ "-","+"])? <FLOAT> )?>
+}
+
+TOKEN: {<IDENTIFIER: (<LETTER>)+(<DIGIT> | <LETTER> | <SPECIALCHAR>)*>}
+TOKEN: {<PATH: (~["(", ")", ";", "\r", " ", "\t", "\n"])+>}
+TOKEN : { <QUOTEDSTRING : "'" (~["'"])* "'"> }
+void parse() throws IOException:
+{
+	Token t1, t2;
+	String val = null;
+}
+
+{
+	(	
+	<EOL>
+	{prompt();}
+	|
+	<CAT>
+	(
+	t1 = GetPath()
+	{processCat(t1.image);}		
+	)+
+	|
+	<CD>
+	(
+		t1 = GetPath()
+		{processCD(t1.image);}
+		|
+		{processCD(null);}
+	)
+	|
+	<COPY>
+	t1 = GetPath()
+	t2 = GetPath()
+	{processCopy(t1.image, t2.image);}
+	|
+	<COPYFROMLOCAL>
+	t1 = GetPath()
+	t2 = GetPath()
+	{processCopyFromLocal(t1.image, t2.image);}	
+	|
+	<COPYTOLOCAL>
+	t1 = GetPath()
+	t2 = GetPath()
+	{processCopyToLocal(t1.image, t2.image);}	
+	|
+	<DEFINE>
+	t1 = <IDENTIFIER>
+	(
+	val = QualifiedFunction()
+	)
+	{processRegisterFunc(t1.image, val);}
+	|
+	<DUMP>
+	t1 = <IDENTIFIER>
+	{processDump(t1.image);}
+	|
+	<DESCRIBE>
+	t1 = <IDENTIFIER>
+	{processDescribe(t1.image);}
+	|
+	<HELP>
+	{printHelp();}
+	|
+	<KILL>
+	t1 = <IDENTIFIER>	
+	{processKill(t1.image);}
+	|
+	<LS>
+	(
+		t1 = GetPath()
+		{processLS(t1.image);}
+		|
+		{processLS(null);}
+	)
+	|
+	<MOVE>
+	t1 = GetPath()
+	t2 = GetPath()
+	{processMove(t1.image, t2.image);}
+	|
+	<MKDIR>
+	t1 = GetPath()
+	{processMkdir(t1.image);}
+	|
+	t1 = <PIG>
+	{processPig(t1.image);}
+	|
+	<PWD>
+	{processPWD();}
+	|
+	<QUIT>
+	{quit();}
+	|
+	<REGISTER>
+	t1 = GetPath()
+	{processRegister(t1.image);}
+	|
+	<REMOVE>
+	(
+		t1 = GetPath()
+		{processRemove(t1.image);}
+	)+	
+	|
+	<SCRIPT_DONE>
+	{quit();}
+	|
+	<SET>
+	(
+		t1 = GetKey()
+		t2 = GetValue()
+		{processSet(t1.image, t2.image);}
+	)
+	|
+	<STORE>
+	t1 = <IDENTIFIER>
+	<INTO>
+	t2 = GetPath()
+	(
+		<USING>
+		val = QualifiedFunction()
+	)?
+	{processStore(t1.image, unquote(t2.image), val);}
+	|
+	<EOF>
+	{quit();}
+	|
+	<SEMICOLON>
+	{}
+	|
+	// handle invalid token
+	skip_invalid_command(EOL)
+	{prompt();}
+	)
+}
+
+String QualifiedFunction()       : {Token t1;StringBuffer s=new StringBuffer();}
+{
+	t1 = GetPath()
+	{s.append(t1.image);}
+	(
+		t1 = <FUNC_ARGS>
+		{s.append(t1.image);}
+	)*
+        {return s.toString();}
+}
+
+Token GetPath() :
+{
+	Token t;
+}
+{
+	(
+	t = <IDENTIFIER>
+	|
+        t = <PATH>
+	|
+	t = GetReserved()
+	)
+	{return t;}
+}
+
+Token GetKey() :
+{	
+	Token t;
+}
+{
+	t = GetPath()
+	{return t;}
+}
+
+Token GetValue() :
+{
+	Token t;
+}
+{
+	(
+	t = GetPath()
+	|
+	t = <QUOTEDSTRING>
+	)
+
+	{return t;}
+}
+
+Token GetReserved () :
+{
+	Token t;
+}
+{
+	(
+	t = <CAT>
+	|
+	t = <CD>
+	|
+	t = <COPY>
+	|
+	t = <COPYFROMLOCAL>
+	|
+	t = <COPYTOLOCAL>
+	|
+	t = <DEFINE>
+	|
+	t = <DUMP>
+	|
+	t = <DESCRIBE>
+	|
+	t = <HELP>
+	|
+	t = <KILL>
+	|
+	t = <LS>
+	|
+	t = <MOVE>
+	|
+	t = <MKDIR>
+	|
+	t = <PWD>
+	|
+	t = <QUIT>
+	|
+	t = <REGISTER>
+	|
+	t = <REMOVE>
+	|
+	t = <SET>
+	|
+	t = <STORE>
+	|
+	t = <INTO>
+	|
+	t = <USING>
+	|
+	t = <SCRIPT_DONE>
+	)
+
+	{return t;}
+}
+
+
+JAVACODE 
+void skip_invalid_command(int kind) 
+{
+	ParseException e = generateParseException();  // generate the exception object.
+	System.out.println(e.toString());  // print the error message
+	Token t = getNextToken();
+
+	while (t.kind != kind)
+		t = getNextToken();
+}