You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ga...@apache.org on 2008/07/31 19:59:30 UTC

svn commit: r681426 - in /incubator/pig/branches/types: ./ src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/ src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/ src/org/apache/pig/builtin/ src/org/a...

Author: gates
Date: Thu Jul 31 10:59:29 2008
New Revision: 681426

URL: http://svn.apache.org/viewvc?rev=681426&view=rev
Log:
PIG-303 Santhosh's patch to address not allowing cast to bytearray.  It also adds loading and storing of complex types to PigStorage.


Added:
    incubator/pig/branches/types/src/org/apache/pig/data/parser/
    incubator/pig/branches/types/src/org/apache/pig/data/parser/TextDataParser.jjt
Modified:
    incubator/pig/branches/types/build.xml
    incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POCast.java
    incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java
    incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PlanPrinter.java
    incubator/pig/branches/types/src/org/apache/pig/builtin/BinStorage.java
    incubator/pig/branches/types/src/org/apache/pig/builtin/TextLoader.java
    incubator/pig/branches/types/src/org/apache/pig/builtin/Utf8StorageConverter.java
    incubator/pig/branches/types/src/org/apache/pig/data/DataType.java
    incubator/pig/branches/types/src/org/apache/pig/data/DefaultAbstractBag.java
    incubator/pig/branches/types/src/org/apache/pig/data/DefaultTuple.java
    incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java
    incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt
    incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestCombiner.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestConversions.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestDataModel.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestPOCast.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidator.java
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/IsNull1.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/Split1.gld
    incubator/pig/branches/types/test/org/apache/pig/test/utils/GenRandomData.java

Modified: incubator/pig/branches/types/build.xml
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/build.xml?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/build.xml (original)
+++ incubator/pig/branches/types/build.xml Thu Jul 31 10:59:29 2008
@@ -68,6 +68,7 @@
     <property name="src.gen.script.parser.dir" value="${src.gen.dir}/org/apache/pig/tools/pigscript/parser" />
     <property name="src.gen.param.parser.dir" value="${src.gen.dir}/org/apache/pig/tools/parameters" />
     <property name="src.gen.dot.parser.dir" value="${test.src.dir}/org/apache/pig/test/utils/dotGraph/parser" />
+    <property name="src.gen.textdata.parser.dir" value="${src.gen.dir}/org/apache/pig/data/parser" />
     <property name="javacc.home" value="${basedir}/lib" />
 
 	<!-- javadoc properties  -->
@@ -101,6 +102,7 @@
         <mkdir dir="${build.classes}" />
         <mkdir dir="${test.build.classes}" />
         <mkdir dir="${src.gen.dot.parser.dir}" />
+        <mkdir dir="${src.gen.textdata.parser.dir}" />
         <tstamp>
             <format property="timestamp" pattern="MMM dd yyyy, HH:mm:ss" />
         </tstamp>
@@ -147,6 +149,8 @@
         <javacc target="${src.dir}/org/apache/pig/tools/parameters/ParamLoader.jj" outputdirectory="${src.gen.param.parser.dir}" javacchome="${javacc.home}" />
         <jjtree target="${test.src.dir}/org/apache/pig/test/utils/dotGraph/Dot.jjt" outputdirectory="${src.gen.dot.parser.dir}" javacchome="${javacc.home}" />
         <javacc target="${src.gen.dot.parser.dir}/Dot.jj" outputdirectory="${src.gen.dot.parser.dir}" javacchome="${javacc.home}" />
+        <jjtree target="${src.dir}/org/apache/pig/data/parser/TextDataParser.jjt" outputdirectory="${src.gen.textdata.parser.dir}/" javacchome="${javacc.home}" />
+        <javacc target="${src.gen.textdata.parser.dir}/TextDataParser.jj" outputdirectory="${src.gen.textdata.parser.dir}" javacchome="${javacc.home}" />
     </target>
 
     <!-- ================================================================== -->

Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POCast.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POCast.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POCast.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POCast.java Thu Jul 31 10:59:29 2008
@@ -577,16 +577,6 @@
     }
     
     @Override
-    public Result getNext(DataByteArray dba) throws ExecException {
-    	String str = null;
-    	Result res = getNext(str);
-    	if(res.returnStatus == POStatus.STATUS_OK) {
-    		res.result = new DataByteArray(((String)res.result).getBytes());
-    	}
-    	return res;
-    }
-    
-    @Override
     public Result getNext(Tuple t) throws ExecException {
     	PhysicalOperator in = inputs.get(0);
     	Byte resultType = in.getResultType();

Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java Thu Jul 31 10:59:29 2008
@@ -160,6 +160,11 @@
 		try {
 			if(result.returnStatus == POStatus.STATUS_OK) {
 				result.result = func.exec((Tuple) result.result);
+                if(resultType == DataType.BYTEARRAY) {
+                    if(DataType.findType(result.result) != DataType.BYTEARRAY) {
+                        result.result = new DataByteArray(result.result.toString().getBytes());
+                    }
+                }
 				return result;
 			}
 			return result;

Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PlanPrinter.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PlanPrinter.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PlanPrinter.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PlanPrinter.java Thu Jul 31 10:59:29 2008
@@ -143,11 +143,12 @@
             sb.append(planString(((POForEach)node).getInputPlans()));
         }
         
-        List<O> predecessors = mPlan.getPredecessors(node);
-        
-        if (predecessors == null)
+        List<O> originalPredecessors = mPlan.getPredecessors(node);
+        if (originalPredecessors == null)
             return sb.toString();
         
+        List<O> predecessors =  new ArrayList<O>(originalPredecessors);
+        
         Collections.sort(predecessors);
         int i = 0;
         for (O pred : predecessors) {

Modified: incubator/pig/branches/types/src/org/apache/pig/builtin/BinStorage.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/builtin/BinStorage.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/builtin/BinStorage.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/builtin/BinStorage.java Thu Jul 31 10:59:29 2008
@@ -19,6 +19,7 @@
 
 import java.io.BufferedOutputStream;
 import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -207,4 +208,108 @@
         // TODO Auto-generated method stub
         
     }
+
+    public byte[] toBytes(DataBag bag) throws IOException {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        try {
+            DataReaderWriter.writeDatum(dos, bag);
+        } catch (Exception ee) {
+            IOException oughtToBeEE = new IOException();
+            ee.initCause(ee);
+            throw oughtToBeEE;
+        }
+        return baos.toByteArray();
+    }
+
+    public byte[] toBytes(String s) throws IOException {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        try {
+            DataReaderWriter.writeDatum(dos, s);
+        } catch (Exception ee) {
+            IOException oughtToBeEE = new IOException();
+            ee.initCause(ee);
+            throw oughtToBeEE;
+        }
+        return baos.toByteArray();
+    }
+
+    public byte[] toBytes(Double d) throws IOException {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        try {
+            DataReaderWriter.writeDatum(dos, d);
+        } catch (Exception ee) {
+            IOException oughtToBeEE = new IOException();
+            ee.initCause(ee);
+            throw oughtToBeEE;
+        }
+        return baos.toByteArray();
+    }
+
+    public byte[] toBytes(Float f) throws IOException {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        try {
+            DataReaderWriter.writeDatum(dos, f);
+        } catch (Exception ee) {
+            IOException oughtToBeEE = new IOException();
+            ee.initCause(ee);
+            throw oughtToBeEE;
+        }
+        return baos.toByteArray();
+    }
+
+    public byte[] toBytes(Integer i) throws IOException {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        try {
+            DataReaderWriter.writeDatum(dos, i);
+        } catch (Exception ee) {
+            IOException oughtToBeEE = new IOException();
+            ee.initCause(ee);
+            throw oughtToBeEE;
+        }
+        return baos.toByteArray();
+    }
+
+    public byte[] toBytes(Long l) throws IOException {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        try {
+            DataReaderWriter.writeDatum(dos, l);
+        } catch (Exception ee) {
+            IOException oughtToBeEE = new IOException();
+            ee.initCause(ee);
+            throw oughtToBeEE;
+        }
+        return baos.toByteArray();
+    }
+
+    public byte[] toBytes(Map<Object, Object> m) throws IOException {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        try {
+            DataReaderWriter.writeDatum(dos, m);
+        } catch (Exception ee) {
+            IOException oughtToBeEE = new IOException();
+            ee.initCause(ee);
+            throw oughtToBeEE;
+        }
+        return baos.toByteArray();
+    }
+
+    public byte[] toBytes(Tuple t) throws IOException {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        try {
+            DataReaderWriter.writeDatum(dos, t);
+        } catch (Exception ee) {
+            IOException oughtToBeEE = new IOException();
+            ee.initCause(ee);
+            throw oughtToBeEE;
+        }
+        return baos.toByteArray();
+    }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/builtin/TextLoader.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/builtin/TextLoader.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/builtin/TextLoader.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/builtin/TextLoader.java Thu Jul 31 10:59:29 2008
@@ -147,4 +147,35 @@
         return null;
     }
 
+    public byte[] toBytes(DataBag bag) throws IOException {
+        throw new IOException("TextLoader does not support conversion from Bag");
+    }
+
+    public byte[] toBytes(String s) throws IOException {
+        return s.getBytes();
+    }
+
+    public byte[] toBytes(Double d) throws IOException {
+        throw new IOException("TextLoader does not support conversion from Double");
+    }
+
+    public byte[] toBytes(Float f) throws IOException {
+        throw new IOException("TextLoader does not support conversion from Float");
+    }
+
+    public byte[] toBytes(Integer i) throws IOException {
+        throw new IOException("TextLoader does not support conversion from Integer");
+    }
+
+    public byte[] toBytes(Long l) throws IOException {
+        throw new IOException("TextLoader does not support conversion from Long");
+    }
+
+    public byte[] toBytes(Map<Object, Object> m) throws IOException {
+        throw new IOException("TextLoader does not support conversion from Map");
+    }
+
+    public byte[] toBytes(Tuple t) throws IOException {
+        throw new IOException("TextLoader does not support conversion from Tuple");
+    }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/builtin/Utf8StorageConverter.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/builtin/Utf8StorageConverter.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/builtin/Utf8StorageConverter.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/builtin/Utf8StorageConverter.java Thu Jul 31 10:59:29 2008
@@ -26,6 +26,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.io.ByteArrayInputStream;
 
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.Log;
@@ -38,7 +39,8 @@
 import org.apache.pig.data.Tuple;
 import org.apache.pig.data.TupleFactory;
 import org.apache.pig.data.BagFactory;
-
+import org.apache.pig.data.parser.ParseException;
+import org.apache.pig.data.parser.TextDataParser;
 
 /**
  * This abstract class provides standard conversions between utf8 encoded data
@@ -53,13 +55,29 @@
 
     private Integer mMaxInt = new Integer(Integer.MAX_VALUE);
     private Long mMaxLong = new Long(Long.MAX_VALUE);
+    private TextDataParser dataParser = null;
         
     public Utf8StorageConverter() {
     }
 
+    private Object parseFromBytes(byte[] b) throws ParseException {
+        ByteArrayInputStream in = new ByteArrayInputStream(b);
+        if(dataParser == null) {
+            dataParser = new TextDataParser(in);
+        } else {
+            dataParser.ReInit(in);
+        }
+        return dataParser.Parse();
+    }
+
     public DataBag bytesToBag(byte[] b) throws IOException {
-        //TODO:FIXME
-        return null;       
+        Object o;
+        try {
+            o = parseFromBytes(b);
+        } catch (ParseException pe) {
+            throw new IOException(pe.getMessage());
+        }
+        return (DataBag)o;
     }
 
     public String bytesToCharArray(byte[] b) throws IOException {
@@ -141,23 +159,28 @@
     }
 
     public Map<Object, Object> bytesToMap(byte[] b) throws IOException {
-        //TODO:FIXME
-        return null;
+        Object o;
+        try {
+            o = parseFromBytes(b);
+        } catch (ParseException pe) {
+            throw new IOException(pe.getMessage());
+        }
+        return (Map<Object, Object>)o;
     }
 
     public Tuple bytesToTuple(byte[] b) throws IOException {
-        return bytesToTuple(b, 0, b.length - 1);
+        Object o;
+        try {
+            o = parseFromBytes(b);
+        } catch (ParseException pe) {
+            throw new IOException(pe.getMessage());
+        }
+        return (Tuple)o;
     }
 
-    private Tuple bytesToTuple(byte[] b, int start, int end) throws IOException {
-        //TODO:FIXME
-        return null;
-    }
-      
 
     public byte[] toBytes(DataBag bag) throws IOException {
-        //TODO:FIXME
-        throw new IOException("Conversion from Bag to bytes not supported");
+        return bag.toString().getBytes();
     }
 
     public byte[] toBytes(String s) throws IOException {
@@ -181,13 +204,11 @@
     }
 
     public byte[] toBytes(Map<Object, Object> m) throws IOException {
-        //TODO:FIXME
-        throw new IOException("Conversion from Map to bytes not supported");
+        return DataType.mapToString(m).getBytes();
     }
 
     public byte[] toBytes(Tuple t) throws IOException {
-        //TODO:FIXME
-        throw new IOException("Conversion from Tuple to bytes not supported");  
+        return t.toString().getBytes();
     }
     
 

Modified: incubator/pig/branches/types/src/org/apache/pig/data/DataType.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/data/DataType.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/data/DataType.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/data/DataType.java Thu Jul 31 10:59:29 2008
@@ -661,4 +661,33 @@
         // else return just ERROR
         return DataType.ERROR ;
     }
+    
+    public static String mapToString(Map<Object, Object> m) {
+        boolean hasNext = false;
+        StringBuilder sb = new StringBuilder();
+        sb.append("[");
+        for(Object o: m.keySet()) {
+            if(hasNext) {
+                sb.append(",");
+            } else {
+                hasNext = true;
+            }
+            sb.append(o.toString());
+            sb.append("#");
+            sb.append(m.get(o).toString());
+        }
+        sb.append("]");
+        return sb.toString();
+    }
+
+    public static boolean equalByteArrays(byte[] lhs, byte[] rhs) {
+        if(lhs == null && rhs == null) return true;
+        if(lhs == null || rhs == null) return false;
+        if(lhs.length != rhs.length) return false;
+        for(int i = 0; i < lhs.length; ++i) {
+            if(lhs[i] != rhs[i]) return false;
+        }
+        return true;
+    }
+        
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/data/DefaultAbstractBag.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/data/DefaultAbstractBag.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/data/DefaultAbstractBag.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/data/DefaultAbstractBag.java Thu Jul 31 10:59:29 2008
@@ -267,7 +267,7 @@
             Tuple t = it.next();
             String s = t.toString();
             sb.append(s);
-            if (it.hasNext()) sb.append(", ");
+            if (it.hasNext()) sb.append(",");
         }
         sb.append('}');
         return sb.toString();

Modified: incubator/pig/branches/types/src/org/apache/pig/data/DefaultTuple.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/data/DefaultTuple.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/data/DefaultTuple.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/data/DefaultTuple.java Thu Jul 31 10:59:29 2008
@@ -207,12 +207,21 @@
         for (Iterator<Object> it = mFields.iterator(); it.hasNext();) {
             Object d = it.next();
             if(d != null) {
-                sb.append(d.toString());
+                if(d instanceof Map) {
+                    sb.append(DataType.mapToString((Map<Object, Object>)d));
+                } else {
+                    sb.append(d.toString());
+                    if(d instanceof Long) {
+                        sb.append("L");
+                    } else if(d instanceof Float) {
+                        sb.append("F");
+                    }
+                }
             } else {
                 sb.append("NULL");
             }
             if (it.hasNext())
-                sb.append(", ");
+                sb.append(",");
         }
         sb.append(')');
         return sb.toString();

Added: incubator/pig/branches/types/src/org/apache/pig/data/parser/TextDataParser.jjt
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/data/parser/TextDataParser.jjt?rev=681426&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/data/parser/TextDataParser.jjt (added)
+++ incubator/pig/branches/types/src/org/apache/pig/data/parser/TextDataParser.jjt Thu Jul 31 10:59:29 2008
@@ -0,0 +1,208 @@
+
+/*
+ * 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.
+ */
+/**
+ * JavaCC file
+ * This file lists the grammar for PIG Latin.
+ * QueryParser program ouputs a ParseTree given a Valid Pig Latin Query
+ */
+options {
+  // Generate non-static functions
+  STATIC = false;
+  // Case is ignored in keywords
+  IGNORE_CASE = true;
+  USER_TOKEN_MANAGER = false;
+  USER_CHAR_STREAM = false;
+}
+
+PARSER_BEGIN(TextDataParser)
+package org.apache.pig.data.parser;
+import java.io.*;
+import java.util.*;
+import org.apache.pig.data.DataType;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.io.*;
+import org.apache.pig.builtin.PigStorage;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.data.TupleFactory;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.BagFactory;
+import org.apache.pig.data.DataBag;
+
+
+public class TextDataParser {
+	private static Log log = LogFactory.getLog(TextDataParser.class);
+}
+
+	
+PARSER_END(TextDataParser)
+
+
+TOKEN :
+{
+ 	< #FLOATINGPOINT: <INTEGER> ( "." <INTEGER> )? | "." <INTEGER> >
+|	<#DIGIT : ["0"-"9"] >
+| 	< #INTEGER: ( <DIGIT> )+ >
+| 	< SIGNEDINTEGER: (["-","+"])? <INTEGER> >
+| 	< LONGINTEGER: <SIGNEDINTEGER> (["l","L"])? >
+|   < DOUBLENUMBER: (["-","+"])? <FLOATINGPOINT> ( ["e","E"] ([ "-","+"])? <FLOATINGPOINT> )?>
+|   < FLOATNUMBER: <DOUBLENUMBER> (["f","F"])? >
+}
+
+TOKEN : { <NULL : "null" > }
+TOKEN : { <STRING : (~[",","(",")","{","}","[","]","#"])* > }
+
+Object Parse() : 
+{
+	Token t1; 
+	String s;
+	Object obj = null;
+	log.trace("Entering Parse");
+}
+{
+	(
+	LOOKAHEAD(AtomDatum()) obj = AtomDatum ()
+|	obj= Datum()
+	)
+	{
+		return obj;
+	}
+}
+
+DataBag Bag() :
+{
+	BagFactory bagFactory = BagFactory.getInstance();
+    DataBag bag = bagFactory.newDefaultBag();
+    Tuple t = null;
+    log.trace("Entering bag");
+}
+{
+ ("(" t = Tuple() {bag.add(t);} ")" ("," "(" t = Tuple() {bag.add(t);} ")" )* )
+ {
+    log.trace("Exiting bag with bag: " + bag);
+    return bag;
+ }
+}
+
+Tuple Tuple() : 
+{
+	Object obj = null;
+	TupleFactory tupleFactory = TupleFactory.getInstance();
+	ArrayList<Object> objList = new ArrayList<Object>(); 
+	log.trace("Entering Tuple");
+}
+{
+	(	
+	(	
+            (
+                obj = Datum() {objList.add(obj);} 
+		        ("," obj = Datum() {objList.add(obj);})* 
+            )
+            |		{}
+	)
+	)
+	{
+		Tuple tuple = tupleFactory.newTuple(objList);
+		log.trace("Exiting Tuple");
+		return tuple;
+	}
+}
+
+Map<Object, Object> Map() :
+{
+	Map<Object, Object> keyValues = new HashMap<Object, Object>();
+	log.trace("Entering Map");
+	
+}
+{
+	( KeyValuePair(keyValues) ("," KeyValuePair(keyValues))* )
+	{
+		log.trace("Exiting Map");
+		return keyValues;
+	}
+}
+
+void KeyValuePair(Map<Object, Object> keyValues) :
+{
+	Object key = null;
+	Object value = null;
+	log.trace("Entering KeyValuePair");
+}
+{
+	(key = AtomDatum() "#" value = Datum())
+	{
+		keyValues.put(key, value);
+		log.trace("Exiting KeyValuePair");
+	}
+	
+}
+
+Object AtomDatum():
+{
+    Object obj = null;
+	Token t;
+	log.trace("Entering AtomDatum");
+}
+{
+	(
+	t = <SIGNEDINTEGER> 
+    {
+        obj = new Integer(Integer.parseInt(t.image));
+    }
+|	t = <LONGINTEGER> 
+    {
+        obj = new Long(Long.parseLong(t.image.substring(0, t.image.length() - 1)));
+    }
+|	t = <FLOATNUMBER> 
+    {
+        obj = new Float(Float.parseFloat(t.image));
+    }
+|	t = <DOUBLENUMBER> 
+    {
+        obj = new Double(Double.parseDouble(t.image));
+    }
+|	t = <NULL> {}
+|	t = <STRING> 
+    {
+        obj = t.image;
+    }
+	)
+	{
+		log.trace("Exiting AtomDatum");
+		return obj;
+	}
+}
+
+Object Datum(): 
+{
+    Object obj = null;
+	log.trace("Entering Datum");
+}
+{
+	(
+	"[" obj = Map() "]"
+|	"{" obj = Bag() "}"
+|	"(" obj = Tuple() ")"
+|	obj = AtomDatum()
+	)
+	{
+		log.trace("Exiting Datum with obj: " + obj);
+		return obj;
+	}
+}

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java Thu Jul 31 10:59:29 2008
@@ -157,11 +157,12 @@
             sb.append(((LOProject)node).getExpression().name());
         }
         
-        List<LogicalOperator> predecessors = mPlan.getPredecessors(node);
-        
-        if (predecessors == null)
+        List<LogicalOperator> originalPredecessors =  mPlan.getPredecessors(node);
+        if (originalPredecessors == null)
             return sb.toString();
         
+        List<LogicalOperator> predecessors =  new ArrayList<LogicalOperator>(originalPredecessors);
+        
         Collections.sort(predecessors);
         int i = 0;
         for (LogicalOperator pred : predecessors) {

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt Thu Jul 31 10:59:29 2008
@@ -2422,29 +2422,24 @@
 |	LOOKAHEAD(TypeSchemaMap()) fs = TypeSchemaMap()
 |	LOOKAHEAD(TypeAtomSchema()) fs = TypeAtomSchema()
 	)
-	//{log.debug("Printing Aliases"); item.printAliases();log.trace("Exiting Schema");return item;}
 	{log.trace("Exiting TypeFieldSchema");return fs;}
 }
 
 Schema.FieldSchema TypeAtomSchema() : 
 {
-	Token t1 = null;
 	byte type = DataType.BYTEARRAY;
 	Schema.FieldSchema fs;
-	log.trace("Entering TypeAtomSchema");
+	log.info("Entering TypeAtomSchema");
 }
 {
-	//(  ( [t1 = <IDENTIFIER> ":"] type = BasicType() )
 	(  ( type = BasicType() )
 		{ 
-			if(null != t1) {
-				log.debug("AtomSchema: " + t1.image);
-				fs = new Schema.FieldSchema(t1.image, type); 
-			} else {
-				fs = new Schema.FieldSchema(null, type); 
-			}
+            if(type == DataType.BYTEARRAY) {
+                throw new ParseException("Cannot cast to bytearray");
+            }
+			fs = new Schema.FieldSchema(null, type); 
 			
-			log.trace("Exiting TypeAtomSchema");
+			log.info("Exiting TypeAtomSchema");
 			return fs;
 		} 
 	)
@@ -2720,9 +2715,9 @@
 }
 {
 	(
-	obj = "[" Map() "]"
-|	obj = "{" Bag() "}"
-|	obj = "(" Tuple() ")"
+	"[" obj = Map() "]"
+|	"{" obj = Bag() "}"
+|	"(" obj = Tuple() ")"
 |	obj = AtomDatum()
 	)
 	{

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java Thu Jul 31 10:59:29 2008
@@ -1543,6 +1543,14 @@
         
         byte inputType = cast.getExpression().getType(); 
         byte expectedType = cast.getType();
+
+
+        if(expectedType == DataType.BYTEARRAY) {
+            String msg = "Cannot cast to bytearray";
+            msgCollector.collect(msg, MessageType.Error) ;
+            throw new VisitorException(msg) ; 
+        }
+        
         Schema.FieldSchema castFs;
         Schema.FieldSchema inputFs;
         try {

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestCombiner.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestCombiner.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestCombiner.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestCombiner.java Thu Jul 31 10:59:29 2008
@@ -65,9 +65,9 @@
         pig.registerQuery("C = foreach B generate flatten(group), COUNT($1);");
         Iterator<Tuple> resultIterator = pig.openIterator("C");
         Tuple tuple = resultIterator.next();
-        assertEquals("(a, b, 2)", tuple.toString());
+        assertEquals("(a,b,2L)", tuple.toString());
         tuple = resultIterator.next();
-        assertEquals("(a, c, 1)", tuple.toString());
+        assertEquals("(a,c,1L)", tuple.toString());
     }
 
     private void loadWithTestLoadFunc(String loadAlias, PigServer pig,

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestConversions.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestConversions.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestConversions.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestConversions.java Thu Jul 31 10:59:29 2008
@@ -17,13 +17,22 @@
  */
 package org.apache.pig.test;
 
+import java.util.Random;
+import java.util.Map;
 import java.io.IOException;
 
 import org.apache.pig.builtin.PigStorage;
+import org.apache.pig.test.utils.GenRandomData;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.pig.data.DataBag;
+import org.apache.pig.data.DataByteArray;
+import org.apache.pig.data.DataType;
 
 import org.junit.Test;
 
 import junit.framework.TestCase;
+import junit.framework.AssertionFailedError;
 
 /**
  * Test class to test conversions from bytes to types
@@ -33,6 +42,8 @@
 public class TestConversions extends TestCase {
 
     PigStorage ps = new PigStorage();
+	Random r = new Random();
+	final int MAX = 10;
     
     @Test
     public  void testBytesToInteger() throws IOException
@@ -130,6 +141,87 @@
         }                        
     }
     
+    @Test
+    public  void testBytesToTuple() throws IOException
+    {
+        for (int i = 0; i < MAX; i++) {
+            Tuple t = GenRandomData.genRandSmallBagTextTuple(r, 1, 100);
+            Tuple convertedTuple = ps.bytesToTuple(t.toString().getBytes());
+            assertTrue(t.equals(convertedTuple));
+        }
+        
+    }
+    
+    @Test
+    public  void testBytesToBag() throws IOException
+    {
+        for (int i = 0; i < MAX; i++) {
+            DataBag b = GenRandomData.genRandFullTupTextDataBag(r,5,100);
+            DataBag convertedBag = ps.bytesToBag(b.toString().getBytes());
+            assertTrue(b.equals(convertedBag));
+        }
+        
+    }
+        
+    @Test
+    public  void testBytesToMap() throws IOException
+    {
+        
+        for (int i = 0; i < MAX; i++) {
+            Map<Object, Object>  m = GenRandomData.genRandObjectMap(r,5);
+            String expectedMapString = DataType.mapToString(m);
+            Map<Object, Object> convertedMap = ps.bytesToMap(expectedMapString.getBytes());
+            assertTrue(m.equals(convertedMap));
+        }
+        
+    }
+
+    @Test
+    public void testIntegerToBytes() throws IOException {
+        Integer i = r.nextInt();
+        assertTrue(DataType.equalByteArrays(i.toString().getBytes(), ps.toBytes(i)));
+    }
+        
+    @Test
+    public void testLongToBytes() throws IOException {
+        Long l = r.nextLong();
+        assertTrue(DataType.equalByteArrays(l.toString().getBytes(), ps.toBytes(l)));
+    }
+        
+    @Test
+    public void testFloatToBytes() throws IOException {
+        Float f = r.nextFloat();
+        assertTrue(DataType.equalByteArrays(f.toString().getBytes(), ps.toBytes(f)));
+    }
+        
+    @Test
+    public void testDoubleToBytes() throws IOException {
+        Double d = r.nextDouble();
+        assertTrue(DataType.equalByteArrays(d.toString().getBytes(), ps.toBytes(d)));
+    }
         
+    @Test
+    public void testCharArrayToBytes() throws IOException {
+        String s = GenRandomData.genRandString(r);
+        assertTrue(s.equals(new String(ps.toBytes(s))));
+    }
+        
+    @Test
+    public void testTupleToBytes() throws IOException {
+        Tuple t = GenRandomData.genRandSmallBagTextTuple(r, 1, 100);
+        //Tuple t = GenRandomData.genRandSmallTuple(r, 100);
+        assertTrue(DataType.equalByteArrays(t.toString().getBytes(), ps.toBytes(t)));
+    }
         
+    @Test
+    public void testBagToBytes() throws IOException {
+        DataBag b = GenRandomData.genRandFullTupTextDataBag(r,5,100);
+        assertTrue(DataType.equalByteArrays(b.toString().getBytes(), ps.toBytes(b)));
+    }
+        
+    @Test
+    public void testMapToBytes() throws IOException {
+        Map<Object, Object>  m = GenRandomData.genRandObjectMap(r,5);
+        assertTrue(DataType.equalByteArrays(DataType.mapToString(m).getBytes(), ps.toBytes(m)));
+    }
 }

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestDataModel.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestDataModel.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestDataModel.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestDataModel.java Thu Jul 31 10:59:29 2008
@@ -227,7 +227,7 @@
     public void testTupleToString() throws Exception {
         Tuple t = giveMeOneOfEach();
 
-        assertEquals("toString", "((3, 3.0), {(4), (mary had a little lamb)}, {hello=world, goodbye=all}, 42, 5000000000, 3.1415927, 2.99792458E8, true, hello, goodbye, NULL)", t.toString());
+        assertEquals("toString", "((3,3.0F),{(4),(mary had a little lamb)},[hello#world,goodbye#all],42,5000000000L,3.1415927F,2.99792458E8,true,hello,goodbye,NULL)", t.toString());
     }
 
     @Test

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestLogToPhyCompiler.java Thu Jul 31 10:59:29 2008
@@ -490,6 +490,13 @@
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         pp.explain(baos);
         String compiledPlan = baos.toString();
+
+        if(generate){
+            FileOutputStream fos = new FileOutputStream("test/org/apache/pig/test/data/GoldenFiles/Limit.gld");
+            fos.write(baos.toByteArray());
+            return;
+        }
+        
         FileInputStream fis = new FileInputStream("test/org/apache/pig/test/data/GoldenFiles/Limit.gld");
         byte[] b = new byte[MAX_SIZE];
         int len = fis.read(b);
@@ -514,12 +521,18 @@
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         pp.explain(baos);
         String compiledPlan = baos.toString();
+
+        if(generate){
+            FileOutputStream fos = new FileOutputStream("test/org/apache/pig/test/data/GoldenFiles/LimitedSort.gld");
+            fos.write(baos.toByteArray());
+            return;
+        }
+        
         FileInputStream fis = new FileInputStream("test/org/apache/pig/test/data/GoldenFiles/LimitedSort.gld");
         byte[] b = new byte[MAX_SIZE];
         int len = fis.read(b);
         String goldenPlan = new String(b, 0, len);
 
-
         System.out.println();
         System.out.println(compiledPlan);
         System.out.println("-------------");

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestLogicalPlanBuilder.java Thu Jul 31 10:59:29 2008
@@ -435,6 +435,38 @@
         public Tuple bytesToTuple(byte[] b) throws IOException {
             return null;
         }        
+
+	    public byte[] toBytes(DataBag bag) throws IOException {
+            return null;
+	    }
+	
+	    public byte[] toBytes(String s) throws IOException {
+            return null;
+	    }
+	
+	    public byte[] toBytes(Double d) throws IOException {
+            return null;
+	    }
+	
+	    public byte[] toBytes(Float f) throws IOException {
+            return null;
+	    }
+	
+	    public byte[] toBytes(Integer i) throws IOException {
+            return null;
+	    }
+	
+	    public byte[] toBytes(Long l) throws IOException {
+            return null;
+	    }
+	
+	    public byte[] toBytes(Map<Object, Object> m) throws IOException {
+            return null;
+	    }
+	
+	    public byte[] toBytes(Tuple t) throws IOException {
+            return null;
+	    }
     }
     
     
@@ -872,7 +904,6 @@
         buildPlan("f = foreach x generate (tuple(int, float))($1/2);");
         buildPlan("g = foreach x generate (tuple())($1/2);");
         buildPlan("h = foreach x generate (chararray)($1/2);");
-        buildPlan("i = foreach x generate (bytearray)($1/2);");
     }
 
     @Test

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java Thu Jul 31 10:59:29 2008
@@ -202,6 +202,38 @@
             return null;
         }
 
+	    public byte[] toBytes(DataBag bag) throws IOException {
+	        return null;
+	    }
+	
+	    public byte[] toBytes(String s) throws IOException {
+	        return s.getBytes();
+	    }
+	
+	    public byte[] toBytes(Double d) throws IOException {
+	        return d.toString().getBytes();
+	    }
+	
+	    public byte[] toBytes(Float f) throws IOException {
+	        return f.toString().getBytes();
+	    }
+	
+	    public byte[] toBytes(Integer i) throws IOException {
+	        return i.toString().getBytes();
+	    }
+	
+	    public byte[] toBytes(Long l) throws IOException {
+	        return l.toString().getBytes();
+	    }
+	
+	    public byte[] toBytes(Map<Object, Object> m) throws IOException {
+	        return m.toString().getBytes();
+	    }
+	
+	    public byte[] toBytes(Tuple t) throws IOException {
+	        return null;
+	    }
+
     }
     @Test
     public void testStoreFunction() throws Throwable {

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestPOCast.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestPOCast.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestPOCast.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestPOCast.java Thu Jul 31 10:59:29 2008
@@ -41,6 +41,8 @@
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POProject;
 import org.apache.pig.impl.plan.PlanException;
 import org.apache.pig.test.utils.GenRandomData;
+import org.apache.pig.builtin.PigStorage;
+import org.apache.pig.builtin.BinStorage;
 import org.junit.Test;
 
 import junit.framework.TestCase;
@@ -61,6 +63,8 @@
 		}
 		
 		POCast op = new POCast(new OperatorKey("", r.nextLong()), -1);
+		LoadFunc load = new TestLoader();
+		op.setLoadFSpec(load.getClass().getName());
 		POProject prj = new POProject(new OperatorKey("", r.nextLong()), -1, 0);
 		PhysicalPlan plan = new PhysicalPlan();
 		plan.add(prj);
@@ -75,7 +79,6 @@
 			Integer i = (Integer) t.get(0);
 			Result res = op.getNext(i);
 			if(res.returnStatus == POStatus.STATUS_OK) {
-				//System.out.println(res.result + " : " + i);
 				assertEquals(i, res.result);
 			}
 		}
@@ -86,7 +89,6 @@
 			Float f = ((Integer)t.get(0)).floatValue();
 			Result res = op.getNext(f);
 			if(res.returnStatus == POStatus.STATUS_OK) {
-//				System.out.println(res.result + " : " + f);
 				assertEquals(f, res.result);
 			}
 		}
@@ -97,7 +99,6 @@
 			Long l = ((Integer)t.get(0)).longValue();
 			Result res = op.getNext(l);
 			if(res.returnStatus == POStatus.STATUS_OK) {
-				//System.out.println(res.result + " : " + l);
 				assertEquals(l, res.result);
 			}
 		}
@@ -108,7 +109,6 @@
 			Double d = ((Integer)t.get(0)).doubleValue();
 			Result res = op.getNext(d);
 			if(res.returnStatus == POStatus.STATUS_OK) {
-				//System.out.println(res.result + " : " + d);
 				assertEquals(d, res.result);
 			}
 		}
@@ -119,7 +119,6 @@
 			String str = ((Integer)t.get(0)).toString();
 			Result res = op.getNext(str);
 			if(res.returnStatus == POStatus.STATUS_OK) {
-				//System.out.println(res.result + " : " + str);
 				assertEquals(str, res.result);
 			}
 		}
@@ -130,7 +129,6 @@
 			DataByteArray dba = new DataByteArray(((Integer)t.get(0)).toString().getBytes());
 			Result res = op.getNext(dba);
 			if(res.returnStatus == POStatus.STATUS_OK) {
-				//System.out.println(res.result + " : " + dba);
 				assertEquals(dba, res.result);
 			}
 		}
@@ -170,6 +168,8 @@
 		}
 		
 		POCast op = new POCast(new OperatorKey("", r.nextLong()), -1);
+		LoadFunc load = new TestLoader();
+		op.setLoadFSpec(load.getClass().getName());
 		POProject prj = new POProject(new OperatorKey("", r.nextLong()), -1, 0);
 		PhysicalPlan plan = new PhysicalPlan();
 		plan.add(prj);
@@ -279,6 +279,8 @@
 		}
 		
 		POCast op = new POCast(new OperatorKey("", r.nextLong()), -1);
+		LoadFunc load = new TestLoader();
+		op.setLoadFSpec(load.getClass().getName());
 		POProject prj = new POProject(new OperatorKey("", r.nextLong()), -1, 0);
 		PhysicalPlan plan = new PhysicalPlan();
 		plan.add(prj);
@@ -388,6 +390,8 @@
 		}
 		
 		POCast op = new POCast(new OperatorKey("", r.nextLong()), -1);
+		LoadFunc load = new TestLoader();
+		op.setLoadFSpec(load.getClass().getName());
 		POProject prj = new POProject(new OperatorKey("", r.nextLong()), -1, 0);
 		PhysicalPlan plan = new PhysicalPlan();
 		plan.add(prj);
@@ -489,6 +493,8 @@
 	@Test
 	public void testStringToOther() throws PlanException, ExecException {
 		POCast op = new POCast(new OperatorKey("", r.nextLong()), -1);
+		LoadFunc load = new TestLoader();
+		op.setLoadFSpec(load.getClass().getName());
 		POProject prj = new POProject(new OperatorKey("", r.nextLong()), -1, 0);
 		PhysicalPlan plan = new PhysicalPlan();
 		plan.add(prj);
@@ -656,6 +662,38 @@
         public Tuple bytesToTuple(byte[] b) throws IOException {
             return null;
         }        
+
+	    public byte[] toBytes(DataBag bag) throws IOException {
+	        return null;
+	    }
+	
+	    public byte[] toBytes(String s) throws IOException {
+	        return s.getBytes();
+	    }
+	
+	    public byte[] toBytes(Double d) throws IOException {
+	        return d.toString().getBytes();
+	    }
+	
+	    public byte[] toBytes(Float f) throws IOException {
+	        return f.toString().getBytes();
+	    }
+	
+	    public byte[] toBytes(Integer i) throws IOException {
+	        return i.toString().getBytes();
+	    }
+	
+	    public byte[] toBytes(Long l) throws IOException {
+	        return l.toString().getBytes();
+	    }
+	
+	    public byte[] toBytes(Map<Object, Object> m) throws IOException {
+	        return null;
+	    }
+	
+	    public byte[] toBytes(Tuple t) throws IOException {
+	        return null;
+	    }
     }
 	
 	@Test
@@ -740,10 +778,7 @@
 			plan.attachInput(t);
 			DataByteArray dba = (DataByteArray) t.get(0);
 			Result res = op.getNext(dba);
-			if(res.returnStatus == POStatus.STATUS_OK) {
-				//System.out.println(res.result + " : " + dba);
-				assertEquals(dba, res.result);
-			}
+			assertEquals(POStatus.STATUS_ERR, res.returnStatus);
 		}
 		
 		{
@@ -782,6 +817,7 @@
 	@Test
 	public void testTupleToOther() throws PlanException, ExecException {
 		POCast op = new POCast(new OperatorKey("", r.nextLong()), -1);
+		op.setLoadFSpec(PigStorage.class.getName());
 		POProject prj = new POProject(new OperatorKey("", r.nextLong()), -1, 0);
 		PhysicalPlan plan = new PhysicalPlan();
 		plan.add(prj);
@@ -889,12 +925,18 @@
 			DataByteArray i = null;
 			Result res = op.getNext(i);
 			assertEquals(POStatus.STATUS_ERR, res.returnStatus);
+
+		    op.setLoadFSpec(BinStorage.class.getName());
+			plan.attachInput(tNew);
+			res = op.getNext(i);
+			assertEquals(POStatus.STATUS_ERR, res.returnStatus);
 		}
 	}
 	
 	@Test
 	public void testBagToOther() throws PlanException, ExecException {
 		POCast op = new POCast(new OperatorKey("", r.nextLong()), -1);
+		op.setLoadFSpec(PigStorage.class.getName());
 		POProject prj = new POProject(new OperatorKey("", r.nextLong()), -1, 0);
 		PhysicalPlan plan = new PhysicalPlan();
 		plan.add(prj);
@@ -984,12 +1026,18 @@
 			DataByteArray i = null;
 			Result res = op.getNext(i);
 			assertEquals(POStatus.STATUS_ERR, res.returnStatus);
+
+		    op.setLoadFSpec(BinStorage.class.getName());
+			plan.attachInput(t);
+			res = op.getNext(i);
+			assertEquals(POStatus.STATUS_ERR, res.returnStatus);
 		}
 	}
 	
 	@Test
 	public void testMapToOther() throws PlanException, ExecException {
 		POCast op = new POCast(new OperatorKey("", r.nextLong()), -1);
+		op.setLoadFSpec(PigStorage.class.getName());
 		POProject prj = new POProject(new OperatorKey("", r.nextLong()), -1, 0);
 		PhysicalPlan plan = new PhysicalPlan();
 		plan.add(prj);
@@ -1069,10 +1117,16 @@
 		
 		{
 			Tuple t = tf.newTuple();
-			t.append(GenRandomData.genRandMap(r, 10));
+			t.append(GenRandomData.genRandObjectMap(r, 10));
+			plan.attachInput(t);
 			DataByteArray i = null;
 			Result res = op.getNext(i);
 			assertEquals(POStatus.STATUS_ERR, res.returnStatus);
+
+		    op.setLoadFSpec(BinStorage.class.getName());
+			plan.attachInput(t);
+			res = op.getNext(i);
+			assertEquals(POStatus.STATUS_ERR, res.returnStatus);
 		}
 	}
 }

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidator.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidator.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidator.java Thu Jul 31 10:59:29 2008
@@ -47,11 +47,11 @@
         constant1.setType(DataType.INTEGER) ;
         LOConst constant2 =  new LOConst(plan, genNewOperatorKey(), 20D) ;
         constant2.setType(DataType.DOUBLE) ;
-        LOConst constant3 =  new LOConst(plan, genNewOperatorKey(), "123") ;
-        constant3.setType(DataType.CHARARRAY) ;
+        LOConst constant3 =  new LOConst(plan, genNewOperatorKey(), 123f) ;
+        constant3.setType(DataType.FLOAT) ;
         
         LOAdd add1 = new LOAdd(plan, genNewOperatorKey(), constant1, constant2) ;
-        LOCast cast1 = new LOCast(plan, genNewOperatorKey(), constant3, DataType.BYTEARRAY) ;
+        LOCast cast1 = new LOCast(plan, genNewOperatorKey(), constant3, DataType.DOUBLE) ;
         LOMultiply mul1 = new LOMultiply(plan, genNewOperatorKey(), add1, cast1) ;
         
         plan.add(constant1) ;
@@ -88,6 +88,50 @@
     }
     
     @Test
+    public void testExpressionTypeCheckingFail1() throws Throwable {
+        LogicalPlan plan = new LogicalPlan() ;
+        LOConst constant1 = new LOConst(plan, genNewOperatorKey(), 10) ;
+        constant1.setType(DataType.INTEGER) ;
+        LOConst constant2 =  new LOConst(plan, genNewOperatorKey(), 20D) ;
+        constant2.setType(DataType.DOUBLE) ;
+        LOConst constant3 =  new LOConst(plan, genNewOperatorKey(), "123") ;
+        constant3.setType(DataType.CHARARRAY) ;
+        
+        LOAdd add1 = new LOAdd(plan, genNewOperatorKey(), constant1, constant2) ;
+        LOCast cast1 = new LOCast(plan, genNewOperatorKey(), constant3, DataType.BYTEARRAY) ;
+        LOMultiply mul1 = new LOMultiply(plan, genNewOperatorKey(), add1, cast1) ;
+        
+        plan.add(constant1) ;
+        plan.add(constant2) ;
+        plan.add(constant3) ;
+        plan.add(cast1) ;
+        plan.add(add1) ;
+        plan.add(mul1) ;
+        
+        plan.connect(constant1, add1) ;
+        plan.connect(constant2, add1) ;
+        plan.connect(add1, mul1) ;
+        plan.connect(constant3, cast1) ;
+        plan.connect(cast1, mul1) ;
+                          
+        CompilationMessageCollector collector = new CompilationMessageCollector() ;
+        TypeCheckingValidator typeValidator = new TypeCheckingValidator() ;
+        try {
+            typeValidator.validate(plan, collector) ;        
+            fail("Exception expected") ;
+        }
+        catch (PlanValidationException pve) {
+            // good
+        }
+        printMessageCollector(collector) ;
+        printTypeGraph(plan) ;
+        
+        if (!collector.hasError()) {
+            throw new Exception("Error during type checking") ;
+        }       
+    }
+
+    @Test
     public void testExpressionTypeChecking2() throws Throwable {
         LogicalPlan plan = new LogicalPlan() ;
         LOConst constant1 = new LOConst(plan, genNewOperatorKey(), 10) ;
@@ -196,11 +240,11 @@
         constant1.setType(DataType.INTEGER) ;
         LOConst constant2 =  new LOConst(plan, genNewOperatorKey(), 20D) ;
         constant2.setType(DataType.DOUBLE) ;
-        LOConst constant3 =  new LOConst(plan, genNewOperatorKey(), "123") ;
-        constant3.setType(DataType.CHARARRAY) ;
+        LOConst constant3 =  new LOConst(plan, genNewOperatorKey(), 123f) ;
+        constant3.setType(DataType.FLOAT) ;
         
         LODivide div1 = new LODivide(plan, genNewOperatorKey(), constant1, constant2) ;
-        LOCast cast1 = new LOCast(plan, genNewOperatorKey(), constant3, DataType.BYTEARRAY) ;
+        LOCast cast1 = new LOCast(plan, genNewOperatorKey(), constant3, DataType.DOUBLE) ;
         LONotEqual notequal1 = new LONotEqual(plan, genNewOperatorKey(), div1, cast1) ;
         
         plan.add(constant1) ;
@@ -238,6 +282,51 @@
     }
     
     @Test
+    public void testExpressionTypeCheckingFail4() throws Throwable {
+        LogicalPlan plan = new LogicalPlan() ;
+        LOConst constant1 = new LOConst(plan, genNewOperatorKey(), 10) ;
+        constant1.setType(DataType.INTEGER) ;
+        LOConst constant2 =  new LOConst(plan, genNewOperatorKey(), 20D) ;
+        constant2.setType(DataType.DOUBLE) ;
+        LOConst constant3 =  new LOConst(plan, genNewOperatorKey(), "123") ;
+        constant3.setType(DataType.CHARARRAY) ;
+        
+        LODivide div1 = new LODivide(plan, genNewOperatorKey(), constant1, constant2) ;
+        LOCast cast1 = new LOCast(plan, genNewOperatorKey(), constant3, DataType.BYTEARRAY) ;
+        LONotEqual notequal1 = new LONotEqual(plan, genNewOperatorKey(), div1, cast1) ;
+        
+        plan.add(constant1) ;
+        plan.add(constant2) ;
+        plan.add(constant3) ;
+        plan.add(div1) ;
+        plan.add(cast1) ;
+        plan.add(notequal1) ;
+        
+        plan.connect(constant1, div1) ;
+        plan.connect(constant2, div1) ;
+        plan.connect(constant3, cast1) ;
+        plan.connect(div1, notequal1) ;
+        plan.connect(cast1, notequal1) ;
+        
+                          
+        CompilationMessageCollector collector = new CompilationMessageCollector() ;
+        TypeCheckingValidator typeValidator = new TypeCheckingValidator() ;
+        try{
+            typeValidator.validate(plan, collector) ;
+            fail("Exception expected") ;
+        }
+        catch (PlanValidationException pve) {
+            // good
+        }
+        printMessageCollector(collector) ;
+        printTypeGraph(plan) ;
+        
+        if (!collector.hasError()) {
+            throw new Exception("Error during type checking") ;
+        }  
+    }
+
+    @Test
     public void testExpressionTypeChecking5() throws Throwable {
         LogicalPlan plan = new LogicalPlan() ;
         LOConst constant1 = new LOConst(plan, genNewOperatorKey(), 10) ;

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/IsNull1.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/IsNull1.gld?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/IsNull1.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/IsNull1.gld Thu Jul 31 10:59:29 2008
@@ -1,20 +1,20 @@
 Filter[tuple] - Test-Plan-Builder-214
 |   |
-|   POIsNull[tuple] - Test-Plan-Builder-216
+|   Not[boolean] - Test-Plan-Builder-217
 |   |
-|   |---Project[bytearray][0] - Test-Plan-Builder-215
+|   |---POIsNull[tuple] - Test-Plan-Builder-216
+|       |
+|       |---Project[bytearray][0] - Test-Plan-Builder-215
 |
 |---Split - Test-Plan-Builder-213
     |
     |---Load(a:org.apache.pig.builtin.PigStorage) - Test-Plan-Builder-212
 
-Filter[tuple] - Test-Plan-Builder-217
+Filter[tuple] - Test-Plan-Builder-218
 |   |
-|   Not[boolean] - Test-Plan-Builder-220
+|   POIsNull[tuple] - Test-Plan-Builder-220
 |   |
-|   |---POIsNull[tuple] - Test-Plan-Builder-219
-|       |
-|       |---Project[bytearray][0] - Test-Plan-Builder-218
+|   |---Project[bytearray][0] - Test-Plan-Builder-219
 |
 |---Split - Test-Plan-Builder-213
     |

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/Split1.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/Split1.gld?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/Split1.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/Split1.gld Thu Jul 31 10:59:29 2008
@@ -1,6 +1,6 @@
 Filter[tuple] - Test-Plan-Builder-196
 |   |
-|   Less Than[boolean] - Test-Plan-Builder-199
+|   Greater Than[boolean] - Test-Plan-Builder-199
 |   |
 |   |---Project[bytearray][0] - Test-Plan-Builder-197
 |   |
@@ -12,7 +12,7 @@
 
 Filter[tuple] - Test-Plan-Builder-200
 |   |
-|   Greater Than[boolean] - Test-Plan-Builder-203
+|   Less Than[boolean] - Test-Plan-Builder-203
 |   |
 |   |---Project[bytearray][0] - Test-Plan-Builder-201
 |   |

Modified: incubator/pig/branches/types/test/org/apache/pig/test/utils/GenRandomData.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/utils/GenRandomData.java?rev=681426&r1=681425&r2=681426&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/utils/GenRandomData.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/utils/GenRandomData.java Thu Jul 31 10:59:29 2008
@@ -42,6 +42,18 @@
         return ret;
     }
     
+    public static Map<Object,Object> genRandObjectMap(Random r, int numEnt) {
+        Map<Object,Object> ret = new HashMap<Object, Object>();
+        if(r==null){
+            ret.put(1, "RANDOM");
+            return ret;
+        }
+        for(int i=0;i<numEnt;i++){
+            ret.put(r.nextInt(), genRandString(r));
+        }
+        return ret;
+    }
+    
     public static String genRandString(Random r){
         if(r==null) return "RANDOM";
         char[] chars = new char[10];
@@ -58,6 +70,11 @@
         return new DataByteArray(bytes);
     }
     
+    public static DataByteArray genRandTextDBA(Random r){
+        if(r==null) return new DataByteArray("RANDOM".getBytes());
+        return new DataByteArray(genRandString(r).getBytes());
+    }
+    
     public static Tuple genRandSmallTuple(Random r, int limit){
         if(r==null){
             Tuple t = new DefaultTuple();
@@ -112,6 +129,30 @@
         return t;
     }
     
+    public static Tuple genRandSmallBagTextTuple(Random r, int num, int limit){
+        if(r==null){
+            Tuple t = new DefaultTuple();
+            t.append("RANDOM");
+            return t;
+        }
+        Tuple t = new DefaultTuple();
+        t.append(genRandSmallTupDataBag(r, num, limit));
+        t.append(new Boolean(r.nextBoolean()).toString());
+        //TODO Fix
+        //The text representation of byte array and char array
+        //cannot be disambiguated without annotation. For now,
+        //the tuples will not contain byte array
+        //t.append(genRandTextDBA(r));
+        t.append(genRandString(r));
+        t.append(r.nextDouble());
+        t.append(r.nextFloat());
+        t.append(r.nextInt());
+        t.append(r.nextLong());
+        t.append(genRandMap(r, num));
+        t.append(genRandSmallTuple(r, 100));
+        return t;
+    }
+    
     public static DataBag genRandFullTupDataBag(Random r, int num, int limit){
         if(r==null) {
             DataBag db = DefaultBagFactory.getInstance().newDefaultBag();
@@ -126,4 +167,19 @@
         }
         return db;
     }
+
+    public static DataBag genRandFullTupTextDataBag(Random r, int num, int limit){
+        if(r==null) {
+            DataBag db = DefaultBagFactory.getInstance().newDefaultBag();
+            Tuple t = new DefaultTuple();
+            t.append("RANDOM");
+            db.add(t);
+            return db;
+        }
+        DataBag db = DefaultBagFactory.getInstance().newDefaultBag();
+        for(int i=0;i<num;i++){
+            db.add(genRandSmallBagTextTuple(r, num, limit));
+        }
+        return db;
+    }
 }