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 2007/11/01 21:48:22 UTC

svn commit: r591143 [3/4] - in /incubator/pig/branches/types: src/org/apache/pig/ src/org/apache/pig/builtin/ src/org/apache/pig/data/ src/org/apache/pig/impl/ src/org/apache/pig/impl/builtin/ src/org/apache/pig/impl/eval/ src/org/apache/pig/impl/eval/...

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/IntermedResult.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/IntermedResult.java?rev=591143&r1=591142&r2=591143&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/IntermedResult.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/IntermedResult.java Thu Nov  1 13:48:16 2007
@@ -25,6 +25,7 @@
 import org.apache.pig.builtin.BinStorage;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.Tuple;
+import org.apache.pig.data.Datum;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileSpec;
 import org.apache.pig.impl.io.PigFile;
@@ -58,7 +59,7 @@
     
     public IntermedResult() {
         executed = true;
-        databag = new DataBag();
+        databag = new DataBag(Datum.DataType.TUPLE);
     }
     
     public IntermedResult(DataBag bag) {

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POCogroup.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POCogroup.java?rev=591143&r1=591142&r2=591143&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POCogroup.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POCogroup.java Thu Nov  1 13:48:16 2007
@@ -121,7 +121,8 @@
 
             boolean done = true;
             for (int i = 0; i < inputs.length; i++) {
-                DataBag b = BagFactory.getInstance().getNewBag();
+                DataBag b =
+					BagFactory.getInstance().getNewBag(Datum.DataType.TUPLE);
 
                 while (sortedInputs[i].size() > 0) {
                     Datum g = sortedInputs[i].get(0)[0];

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/PORead.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/PORead.java?rev=591143&r1=591142&r2=591143&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/PORead.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/PORead.java Thu Nov  1 13:48:16 2007
@@ -22,6 +22,7 @@
 
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.Tuple;
+import org.apache.pig.data.Datum;
 
 
 public class PORead extends PhysicalOperator {
@@ -30,7 +31,7 @@
 	 */
 	private static final long serialVersionUID = 1L;
 	DataBag             bag;
-    Iterator<Tuple> it;
+    Iterator<Datum> it;
 
     public PORead(DataBag bagIn, int outputType) {
     	super(outputType);
@@ -54,7 +55,7 @@
     @Override
 	public Tuple getNext() throws IOException {
         if (it.hasNext())
-            return it.next();
+            return (Tuple)it.next();
         else
             return null;
     }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POSort.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POSort.java?rev=591143&r1=591142&r2=591143&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POSort.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POSort.java Thu Nov  1 13:48:16 2007
@@ -24,13 +24,14 @@
 import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.Tuple;
+import org.apache.pig.data.Datum;
 import org.apache.pig.impl.eval.EvalSpec;
 
 
 public class POSort extends PhysicalOperator {
 	static final long serialVersionUID = 1L; 
 	EvalSpec sortSpec;
-	transient Iterator<Tuple> iter;
+	transient Iterator<Datum> iter;
 	
 	
 	public POSort(EvalSpec sortSpec, int outputType) {
@@ -43,7 +44,8 @@
 	public boolean open(boolean continueFromLast) throws IOException {
 		if (!super.open(continueFromLast))
 			return false;
-		DataBag bag = BagFactory.getInstance().getNewBag();
+		DataBag bag =
+			BagFactory.getInstance().getNewBag(Datum.DataType.TUPLE);
 		
 		bag.sort(sortSpec);
 		Tuple t;
@@ -57,7 +59,7 @@
 	@Override
 	public Tuple getNext() throws IOException {
 		if (iter.hasNext())
-			return iter.next();
+			return (Tuple)iter.next();
 		else
 			return null;
 	}

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POStore.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POStore.java?rev=591143&r1=591142&r2=591143&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POStore.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POStore.java Thu Nov  1 13:48:16 2007
@@ -22,6 +22,7 @@
 import org.apache.pig.StoreFunc;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.Tuple;
+import org.apache.pig.data.Datum;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileSpec;
 import org.apache.pig.impl.io.PigFile;
@@ -62,7 +63,7 @@
     @Override
 	public Tuple getNext() throws IOException {
         // get all tuples from input, and store them.
-        DataBag b = new DataBag();
+        DataBag b = new DataBag(Datum.DataType.TUPLE);
         Tuple t;
         while ((t = (Tuple) inputs[0].getNext()) != null) {
             b.add(t);

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/PhysicalPlan.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/PhysicalPlan.java?rev=591143&r1=591142&r2=591143&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/PhysicalPlan.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/PhysicalPlan.java Thu Nov  1 13:48:16 2007
@@ -21,6 +21,7 @@
 import java.util.Map;
 
 import org.apache.pig.data.DataBag;
+import org.apache.pig.data.Datum;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.LogicalPlan;
 
@@ -33,7 +34,7 @@
     }
 
     public DataBag exec(boolean continueFromLast) throws IOException {
-        DataBag results = new DataBag();
+        DataBag results = new DataBag(Datum.DataType.TUPLE);
 
         root.open(continueFromLast);
         Tuple t;

Added: incubator/pig/branches/types/src/org/apache/pig/impl/util/PigLogger.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/util/PigLogger.java?rev=591143&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/util/PigLogger.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/util/PigLogger.java Thu Nov  1 13:48:16 2007
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.impl.util;
+
+import org.apache.log4j.Logger;
+import org.apache.log4j.Level;
+import org.apache.log4j.ConsoleAppender;
+import org.apache.log4j.PatternLayout;
+
+public class PigLogger 
+{
+
+private static Logger mLogger = null;
+private static boolean mHaveSetAppenders = false;
+
+/**
+ * Get an instance of the underlying log4j logger.  This first makes sure
+ * the PigLogger is initialized and then returns the underlying logger.
+ */ 
+public static Logger getLogger()
+{
+	if (mLogger == null) {
+		mLogger = Logger.getLogger("org.apache.pig");
+		mLogger.setAdditivity(false);
+	}
+	return mLogger;
+}
+
+/**
+ * Set up a log appender for the junit tests, this way they cn write out log
+ * messages.
+ */
+public static void setAppenderForJunit()
+{
+	if (!mHaveSetAppenders) {
+		Logger log = getLogger();
+		log.setLevel(Level.INFO);
+		ConsoleAppender screen = new ConsoleAppender(new PatternLayout());
+		screen.setThreshold(Level.INFO);
+		screen.setTarget(ConsoleAppender.SYSTEM_ERR);
+		log.addAppender(screen);
+		mHaveSetAppenders = true;
+	}
+}
+
+
+}

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java?rev=591143&r1=591142&r2=591143&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestBuiltin.java Thu Nov  1 13:48:16 2007
@@ -34,11 +34,7 @@
 import org.apache.pig.EvalFunc;
 import org.apache.pig.StoreFunc;
 import org.apache.pig.builtin.*;
-import org.apache.pig.data.BagFactory;
-import org.apache.pig.data.DataAtom;
-import org.apache.pig.data.DataBag;
-import org.apache.pig.data.DataMap;
-import org.apache.pig.data.Tuple;
+import org.apache.pig.data.*;
 import org.apache.pig.PigServer.ExecType;
 import org.apache.pig.impl.builtin.ShellBagEvalFunc;
 import org.apache.pig.impl.io.FileLocalizer;
@@ -95,7 +91,7 @@
         count.exec(tup,output);
         assertTrue(output.numval() == 0);
         
-        map.put("a", new DataAtom("a"));
+        map.put("a", "a");
 
         assertFalse(isEmpty.exec(tup));
         count.exec(tup,output);
@@ -176,7 +172,8 @@
         t2.setField(0,a);
         Tuple t3 = new Tuple(1);
         t3.setField(0, b);
-        DataBag bag = BagFactory.getInstance().getNewBigBag();
+        DataBag bag =
+			BagFactory.getInstance().getNewBigBag(Datum.DataType.TUPLE);
         bag.add(t2);
         bag.add(t3);
         Tuple t4 = new Tuple(2);
@@ -195,7 +192,8 @@
         t6.setField(0,c);
         Tuple t7 = new Tuple(1);
         t7.setField(0, d);
-        DataBag bag2 = BagFactory.getInstance().getNewBigBag();    
+        DataBag bag2 =
+			BagFactory.getInstance().getNewBigBag(Datum.DataType.TUPLE);    
         for(int i = 0; i < 10; i ++) {
             bag2.add(t6);
             bag2.add(t7);
@@ -319,8 +317,10 @@
     	for (int i=0; i< numTimes; i++){
     		Tuple t = iter.next();
     		
-    		assertEquals(i+"AA", t.getBagField(0).content().next().getAtomField(0).strval());
-    		assertEquals(i+"BB", t.getBagField(1).content().next().getAtomField(0).strval());
+			Tuple t0 = (Tuple)t.getBagField(0).content().next();
+			Tuple t1 = (Tuple)t.getBagField(1).content().next();
+    		assertEquals(i+"AA", t0.getAtomField(0).strval());
+    		assertEquals(i+"BB", t1.getAtomField(0).strval());
     		
     	}
     	

Added: incubator/pig/branches/types/test/org/apache/pig/test/TestDataBag.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestDataBag.java?rev=591143&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestDataBag.java (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestDataBag.java Thu Nov  1 13:48:16 2007
@@ -0,0 +1,714 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Iterator;
+
+import org.junit.Test;
+
+import org.apache.pig.data.*;
+
+/**
+ * This class will exercise the data bag data type.
+ * 
+ * @author gates
+ */
+public class TestDataBag extends junit.framework.TestCase
+{
+
+public void testDefaultConstructor() throws Exception
+{
+	DataBag bag = new DataBag(Datum.DataType.INT);
+
+	assertEquals("getType", Datum.DataType.BAG, bag.getType());
+	assertFalse("is null", bag.isNull());
+	assertTrue("bag of ints", bag.bagOf() == Datum.DataType.INT);
+
+	assertEquals("Default constructor size before", 0, bag.size());
+	DataInteger val = new DataInteger(42);
+
+	bag.add(val);
+	assertEquals("Default constructor size after", 1, bag.size());
+
+	Iterator<Datum> i = bag.content();
+	Datum d = i.next();
+
+	assertTrue("should be an integer", d.getType() == Datum.DataType.INT);
+	assertNotNull("get with entry in bag", d);
+	assertEquals("value of val", 42, ((DataInteger)d).get());
+}
+
+public void testListConstructor() throws Exception
+{
+	List<Datum> list = new ArrayList<Datum>();
+	list.add(new DataInteger(10));
+	list.add(new DataInteger(11));
+	list.add(new DataInteger(9));
+
+	DataBag bag = new DataBag(list);
+
+	assertEquals("list construct size", 3L, bag.size());
+
+	Iterator<Datum> i = bag.content();
+	Datum d = i.next();
+	assertNotNull("get first entry in bag", d);
+	assertTrue("should be an integer", d.getType() == Datum.DataType.INT);
+	assertEquals("first value of val", 10, ((DataInteger)d).get());
+	d = i.next();
+	assertNotNull("get second entry in bag", d);
+	assertTrue("should be an integer", d.getType() == Datum.DataType.INT);
+	assertEquals("second value of val", 11, ((DataInteger)d).get());
+	d = i.next();
+	assertNotNull("get third entry in bag", d);
+	assertTrue("should be an integer", d.getType() == Datum.DataType.INT);
+	assertEquals("third value of val", 9, ((DataInteger)d).get());
+	assertFalse("bag should be exhausted now", i.hasNext());
+
+	bag.add(new DataInteger(4));
+	i = bag.content();
+	d = i.next();
+	d = i.next();
+	d = i.next();
+	d = i.next();
+	assertNotNull("get fourth entry in bag", d);
+	assertTrue("should be an integer", d.getType() == Datum.DataType.INT);
+	assertEquals("fourth value of val", 4, ((DataInteger)d).get());
+	assertFalse("bag should be exhausted now", i.hasNext());
+}
+
+
+public void testBigBag() throws Exception
+{
+	DataBag bag = new DataBag(Datum.DataType.INT);
+
+	for (int i = 0; i < 10000; i++) {
+		bag.add(new DataInteger(i));
+	}
+
+	assertEquals("big size after loading", 10000, bag.size());
+
+	Iterator<Datum> i = bag.content();
+	for (int j = 0; j < 10000; j++) {
+		assertTrue("should still have data", i.hasNext());
+		Datum val = i.next();
+		assertTrue("should be an integer", val.getType() == Datum.DataType.INT);
+		assertEquals("value of val", j, ((DataInteger)val).get());
+	}
+	assertFalse("bag should be exhausted now", i.hasNext());
+}
+
+public void testToString() throws Exception
+{
+	DataBag bag = new DataBag(Datum.DataType.INT);
+
+	bag.add(new DataInteger(1));
+	bag.add(new DataInteger(1));
+	bag.add(new DataInteger(3));
+
+	assertEquals("toString", "{1, 1, 3}", bag.toString());
+}
+
+public void testEquals() throws Exception
+{
+	DataBag bag1 = new DataBag(Datum.DataType.INT);
+	DataBag bag2 = new DataBag(Datum.DataType.INT);
+
+	bag1.add(new DataInteger(3));
+	bag2.add(new DataInteger(3));
+
+	assertFalse("different object", bag1.equals(new String()));
+
+	assertTrue("same data", bag1.equals(bag2));
+
+	bag2 = new DataBag(Datum.DataType.INT);
+	bag2.add(new DataInteger(4));
+	assertFalse("different data", bag1.equals(bag2));
+
+	bag2 = new DataBag(Datum.DataType.INT);
+	bag2.add(new DataInteger(3));
+	bag2.add(new DataInteger(3));
+	assertFalse("different size", bag1.equals(bag2));
+
+	bag2 = new DataBag(Datum.DataType.LONG);
+	bag2.add(new DataLong(3));
+	assertFalse("different type of bag", bag1.equals(bag2));
+}
+
+public void testCompareTo() throws Exception
+{
+	DataBag bag1 = new DataBag(Datum.DataType.INT);
+	DataBag bag2 = new DataBag(Datum.DataType.INT);
+
+	bag1.add(new DataInteger(3));
+	bag2.add(new DataInteger(3));
+
+	assertEquals("different object less than", -1, bag1.compareTo(new String()));
+
+	Tuple t = new Tuple();
+	assertTrue("less than tuple", bag1.compareTo(t) < 0);
+	DataMap map = new DataMap();
+	assertTrue("less than map", bag1.compareTo(map) < 0);
+	DataLong l = new DataLong();
+	assertTrue("less than long", bag1.compareTo(l) < 0);
+	DataFloat f = new DataFloat();
+	assertTrue("less than float", bag1.compareTo(f) < 0);
+	DataDouble d = new DataDouble();
+	assertTrue("less than double", bag1.compareTo(d) < 0);
+	DataUnknown unk = new DataUnknown();
+	assertTrue("less than unknown", bag1.compareTo(unk) < 0);
+	DataCharArrayUtf16 utf16 = new DataCharArrayUtf16();
+	assertTrue("less than utf16", bag1.compareTo(utf16) < 0);
+
+	assertEquals("same data equal", 0,  bag1.compareTo(bag2));
+
+	bag2 = new DataBag(Datum.DataType.INT);
+	bag2.add(new DataInteger(2));
+	assertEquals("greater than bag with lesser value", 1, bag1.compareTo(bag2));
+
+	bag2 = new DataBag(Datum.DataType.INT);
+	bag2.add(new DataInteger(4));
+	assertEquals("less than bag with greater value", -1, bag1.compareTo(bag2));
+
+	bag2 = new DataBag(Datum.DataType.INT);
+	bag2.add(new DataInteger(3));
+	bag2.add(new DataInteger(4));
+	assertEquals("less than bigger bag", -1, bag1.compareTo(bag2));
+
+	bag2 = new DataBag(Datum.DataType.INT);
+	assertEquals("greater than smaller bag", 1, bag1.compareTo(bag2));
+
+	bag2 = new DataBag(Datum.DataType.LONG);
+	bag2.add(new DataLong(3));
+	assertEquals("different type of bag", -1, bag1.compareTo(bag2));
+}
+
+
+public void testWriteReadUnknown() throws Exception
+{
+	DataBag before = new DataBag(Datum.DataType.UNKNOWN);
+
+	String s = new String("zzz");
+	before.add(new DataUnknown(s.getBytes()));
+	s = new String("yyy");
+	before.add(new DataUnknown(s.getBytes()));
+	s = new String("xxx");
+	before.add(new DataUnknown(s.getBytes()));
+
+	File file = null;
+	file = File.createTempFile("DataBagUnknown", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataBag", a instanceof DataBag);
+
+	DataBag after = (DataBag)a;
+
+	assertTrue("bag of unknowns", after.bagOf() == Datum.DataType.UNKNOWN);
+	assertEquals("after read, size", 3, after.size()); 
+
+	Iterator<Datum> j = after.content();
+
+	Datum valAfter = j.next();
+	assertTrue("should be an unknown",
+		valAfter.getType() == Datum.DataType.UNKNOWN);
+	for (int i = 0; i < 3; i++) {
+		assertEquals("value of valAfter", (byte)0x7a,
+			((DataUnknown)valAfter).get()[i]);
+	}
+
+	valAfter = j.next();
+	assertTrue("should be an unknown",
+		valAfter.getType() == Datum.DataType.UNKNOWN);
+	for (int i = 0; i < 3; i++) {
+		assertEquals("value of valAfter", (byte)0x79,
+			((DataUnknown)valAfter).get()[i]);
+	}
+
+	valAfter = j.next();
+	assertTrue("should be an unknown",
+		valAfter.getType() == Datum.DataType.UNKNOWN);
+	for (int i = 0; i < 3; i++) {
+		assertEquals("value of valAfter", (byte)0x78,
+			((DataUnknown)valAfter).get()[i]);
+	}
+
+	assertFalse("should have read all values in bag", j.hasNext());
+	
+	file.delete();
+}
+
+public void testWriteReadInt() throws Exception
+{
+	DataBag before = new DataBag(Datum.DataType.INT);
+
+	before.add(new DataInteger(99));
+	before.add(new DataInteger(-98));
+	before.add(new DataInteger(97));
+
+	File file = null;
+	file = File.createTempFile("DataBagInteger", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataBag", a instanceof DataBag);
+
+	DataBag after = (DataBag)a;
+
+	assertTrue("bag of ints", after.bagOf() == Datum.DataType.INT);
+
+	assertEquals("after read, size", 3, after.size()); 
+
+	Iterator<Datum> j = after.content();
+
+	Datum val = j.next();
+	assertTrue("should be an integer", val.getType() == Datum.DataType.INT);
+	assertEquals("value of valAfter", 99, ((DataInteger)val).get());
+
+	val = j.next();
+	assertTrue("should be an integer", val.getType() == Datum.DataType.INT);
+	assertEquals("value of valAfter2", -98, ((DataInteger)val).get());
+
+	val = j.next();
+	assertTrue("should be an integer", val.getType() == Datum.DataType.INT);
+	assertEquals("value of valAfter", 97, ((DataInteger)val).get());
+
+	assertFalse("should have read all values in bag", j.hasNext());
+		
+	file.delete();
+}
+
+public void testWriteReadLong() throws Exception
+{
+	DataBag before = new DataBag(Datum.DataType.LONG);
+
+	before.add(new DataLong(99000000000L));
+	before.add(new DataLong(-98L));
+	before.add(new DataLong(97L));
+
+	File file = null;
+	file = File.createTempFile("DataBagLong", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataBag", a instanceof DataBag);
+
+	DataBag after = (DataBag)a;
+
+	assertTrue("bag of longs", after.bagOf() == Datum.DataType.LONG);
+	assertEquals("after read, size", 3, after.size()); 
+
+	Iterator<Datum> j = after.content();
+
+	Datum val = j.next();
+	assertTrue("should be a long", val.getType() == Datum.DataType.LONG);
+	assertEquals("value of valAfter", 99000000000L, ((DataLong)val).get());
+
+	val = j.next();
+	assertTrue("should be a long", val.getType() == Datum.DataType.LONG);
+	assertEquals("value of valAfter2", -98L, ((DataLong)val).get());
+
+	val = j.next();
+	assertTrue("should be a long", val.getType() == Datum.DataType.LONG);
+	assertEquals("value of valAfter", 97L, ((DataLong)val).get());
+
+	assertFalse("should have read all values in bag", j.hasNext());
+		
+	file.delete();
+}
+
+public void testWriteReadFloat() throws Exception
+{
+	DataBag before = new DataBag(Datum.DataType.FLOAT);
+
+	before.add(new DataFloat(3.2e32f));
+	before.add(new DataFloat(-9.929292e-29f));
+	before.add(new DataFloat(97.0f));
+
+	File file = null;
+	file = File.createTempFile("DataBagFloat", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataBag", a instanceof DataBag);
+
+	DataBag after = (DataBag)a;
+
+	assertTrue("bag of floats", after.bagOf() == Datum.DataType.FLOAT);
+	assertEquals("after read, size", 3, after.size()); 
+
+	Iterator<Datum> j = after.content();
+
+	Datum val = j.next();
+	assertTrue("should be a float", val.getType() == Datum.DataType.FLOAT);
+	assertEquals("value of valAfter", 3.2e32f, ((DataFloat)val).get());
+
+	val = j.next();
+	assertTrue("should be a float", val.getType() == Datum.DataType.FLOAT);
+	assertEquals("value of valAfter2", -9.929292e-29f, ((DataFloat)val).get());
+
+	val = j.next();
+	assertTrue("should be a float", val.getType() == Datum.DataType.FLOAT);
+	assertEquals("value of valAfter", 97.0f, ((DataFloat)val).get());
+
+	assertFalse("should have read all values in bag", j.hasNext());
+		
+	file.delete();
+}
+
+public void testWriteReadDouble() throws Exception
+{
+	DataBag before = new DataBag(Datum.DataType.DOUBLE);
+
+	before.add(new DataDouble(3.2e132));
+	before.add(new DataDouble(-9.929292e-129));
+	before.add(new DataDouble(97.0));
+
+	File file = null;
+	file = File.createTempFile("DataBagDouble", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataBag", a instanceof DataBag);
+
+	DataBag after = (DataBag)a;
+
+	assertTrue("bag of double", after.bagOf() == Datum.DataType.DOUBLE);
+	assertEquals("after read, size", 3, after.size()); 
+
+	Iterator<Datum> j = after.content();
+
+	Datum val = j.next();
+	assertTrue("should be a double", val.getType() == Datum.DataType.DOUBLE);
+	assertEquals("value of valAfter", 3.2e132, ((DataDouble)val).get());
+
+	val = j.next();
+	assertTrue("should be a double", val.getType() == Datum.DataType.DOUBLE);
+	assertEquals("value of valAfter2", -9.929292e-129, ((DataDouble)val).get());
+
+	val = j.next();
+	assertTrue("should be a double", val.getType() == Datum.DataType.DOUBLE);
+	assertEquals("value of valAfter", 97.0, ((DataDouble)val).get());
+
+	assertFalse("should have read all values in bag", j.hasNext());
+		
+	file.delete();
+}
+
+public void testWriteReadUtf16() throws Exception
+{
+	DataBag before = new DataBag(Datum.DataType.CHARARRAY);
+
+	before.add(new DataCharArrayUtf16("zzz"));
+	before.add(new DataCharArrayUtf16("yyy"));
+	before.add(new DataCharArrayUtf16("xxx"));
+
+	File file = null;
+	file = File.createTempFile("DataBagUtf16", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataBag", a instanceof DataBag);
+
+	DataBag after = (DataBag)a;
+
+	assertTrue("bag of chararray", after.bagOf() == Datum.DataType.CHARARRAY);
+	assertEquals("after read, size", 3, after.size()); 
+
+	Iterator<Datum> j = after.content();
+
+	Datum val = j.next();
+	assertTrue("should be a chararray", val.getType() == Datum.DataType.CHARARRAY);
+	assertTrue("encoding should be utf16",
+		((DataCharArray)val).getEncoding() == DataCharArray.Encoding.UTF16);
+	assertEquals("value of valAfter", "zzz", ((DataCharArrayUtf16)val).get());
+
+	val = j.next();
+	assertTrue("should be a chararray", val.getType() == Datum.DataType.CHARARRAY);
+	assertTrue("encoding should be utf16",
+		((DataCharArray)val).getEncoding() == DataCharArray.Encoding.UTF16);
+	assertEquals("value of valAfter2", "yyy", ((DataCharArrayUtf16)val).get());
+
+	val = j.next();
+	assertTrue("should be a chararray", val.getType() == Datum.DataType.CHARARRAY);
+	assertTrue("encoding should be utf16",
+		((DataCharArray)val).getEncoding() == DataCharArray.Encoding.UTF16);
+	assertEquals("value of valAfter", "xxx", ((DataCharArrayUtf16)val).get());
+
+	assertFalse("should have read all values in bag", j.hasNext());
+		
+	file.delete();
+}
+
+public void testWriteReadNone() throws Exception
+{
+	DataBag before = new DataBag(Datum.DataType.CHARARRAY);
+
+	String s = new String("zzz");
+	before.add(new DataCharArrayNone(s.getBytes()));
+	s = new String("yyy");
+	before.add(new DataCharArrayNone(s.getBytes()));
+	s = new String("xxx");
+	before.add(new DataCharArrayNone(s.getBytes()));
+
+	File file = null;
+	file = File.createTempFile("DataBagCharArrayNone", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataBag", a instanceof DataBag);
+
+	DataBag after = (DataBag)a;
+
+	assertTrue("bag of chararray", after.bagOf() == Datum.DataType.CHARARRAY);
+	assertEquals("after read, size", 3, after.size()); 
+
+	Iterator<Datum> j = after.content();
+
+	Datum valAfter = j.next();
+	assertTrue("should be a chararray", valAfter.getType() == Datum.DataType.CHARARRAY);
+	assertTrue("encoding should be none",
+		((DataCharArray)valAfter).getEncoding() == DataCharArray.Encoding.NONE);
+	for (int i = 0; i < 3; i++) {
+		assertEquals("value of valAfter", (byte)0x7a,
+			((DataCharArrayNone)valAfter).get()[i]);
+	}
+
+	valAfter = j.next();
+	assertTrue("should be a chararray", valAfter.getType() == Datum.DataType.CHARARRAY);
+	assertTrue("encoding should be none",
+		((DataCharArray)valAfter).getEncoding() == DataCharArray.Encoding.NONE);
+	for (int i = 0; i < 3; i++) {
+		assertEquals("value of valAfter", (byte)0x79,
+			((DataCharArrayNone)valAfter).get()[i]);
+	}
+
+	valAfter = j.next();
+	assertTrue("should be a chararray", valAfter.getType() == Datum.DataType.CHARARRAY);
+	assertTrue("encoding should be none",
+		((DataCharArray)valAfter).getEncoding() == DataCharArray.Encoding.NONE);
+	for (int i = 0; i < 3; i++) {
+		assertEquals("value of valAfter", (byte)0x78,
+			((DataCharArrayNone)valAfter).get()[i]);
+	}
+
+	assertFalse("should have read all values in bag", j.hasNext());
+	
+	file.delete();
+}
+
+public void testWriteReadMap() throws Exception
+{
+	DataBag before = new DataBag(Datum.DataType.MAP);
+
+	DataMap map = new DataMap();
+
+	DataInteger key = new DataInteger(1);
+	Datum val = new DataInteger(99);
+	map.put(key, val);
+
+	before.add(map);
+
+	File file = null;
+	file = File.createTempFile("DataBagCharArrayNone", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataBag", a instanceof DataBag);
+
+	DataBag after = (DataBag)a;
+
+	assertTrue("bag of maps", after.bagOf() == Datum.DataType.MAP);
+	assertEquals("after read, size", 1, after.size()); 
+
+	Iterator<Datum> j = after.content();
+
+	Datum v = j.next();
+	assertTrue("valAfter should be a map", v.getType() == Datum.DataType.MAP);
+	DataMap valAfter = (DataMap)v;
+
+	assertEquals("valAfter size", 1L, valAfter.size());
+
+	DataInteger nosuch = new DataInteger(-1);
+	Datum d = valAfter.get(nosuch);
+	assertTrue("after read, no such key", d.isNull());
+
+	Datum mapValAfter = valAfter.get(key);
+	assertTrue("mapValAfter isa integer", mapValAfter instanceof DataInteger);
+	assertEquals("value of valAfter", 99, ((DataInteger)mapValAfter).get());
+
+	assertFalse("should have read all values in bag", j.hasNext());
+	
+	file.delete();
+}
+
+public void testWriteReadTuple() throws Exception
+{
+	DataBag before = new DataBag(Datum.DataType.TUPLE);
+
+	Tuple t = new Tuple(1);
+	t.setField(0, new DataInteger(1));
+	before.add(t);
+
+	File file = null;
+	file = File.createTempFile("DataBagCharArrayNone", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataBag", a instanceof DataBag);
+
+	DataBag after = (DataBag)a;
+
+	assertTrue("bag of tuples", after.bagOf() == Datum.DataType.TUPLE);
+	assertEquals("after read, size", 1, after.size()); 
+
+	Iterator<Datum> j = after.content();
+
+	Datum v = j.next();
+	assertTrue("valAfter should be a tuple",
+		v.getType() == Datum.DataType.TUPLE);
+
+	Tuple valAfter = (Tuple)v;
+
+	assertEquals("valAfter size", 1L, valAfter.size());
+
+	Datum tupleValAfter = valAfter.getField(0);
+	assertTrue("tupleValAfter isa integer", tupleValAfter instanceof DataInteger);
+	assertEquals("value of valAfter", 1, ((DataInteger)tupleValAfter).get());
+
+	assertFalse("should have read all values in bag", j.hasNext());
+	
+	file.delete();
+}
+
+public void testWriteReadBag() throws Exception
+{
+	DataBag before = new DataBag(Datum.DataType.BAG);
+
+	DataBag b = new DataBag(Datum.DataType.INT);
+	b.add(new DataInteger(2));
+	before.add(b);
+
+	File file = null;
+	file = File.createTempFile("DataBagCharArrayNone", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataBag", a instanceof DataBag);
+
+	DataBag after = (DataBag)a;
+
+	assertTrue("bag of bags", after.bagOf() == Datum.DataType.BAG);
+	assertEquals("after read, size", 1, after.size()); 
+
+	Iterator<Datum> j = after.content();
+
+	Datum v = j.next();
+	assertTrue("valAfter should be a bag", v.getType() == Datum.DataType.BAG);
+	DataBag valAfter = (DataBag)v;
+
+	assertEquals("valAfter size", 1L, valAfter.size());
+
+	Iterator<Datum> k = valAfter.content();
+	Datum w = k.next();
+	assertTrue("bagValAfter should be an integer",
+		w.getType() == Datum.DataType.INT);
+	DataInteger bagValAfter = (DataInteger)w;
+
+	assertEquals("value of valAfter", 2, bagValAfter.get());
+
+	assertFalse("should have read all values in inner bag", k.hasNext());
+	assertFalse("should have read all values in bag", j.hasNext());
+	
+	file.delete();
+}
+
+}
+
+
+ 

Added: incubator/pig/branches/types/test/org/apache/pig/test/TestDataCharArrayNone.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestDataCharArrayNone.java?rev=591143&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestDataCharArrayNone.java (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestDataCharArrayNone.java Thu Nov  1 13:48:16 2007
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileInputStream;
+
+import org.junit.Test;
+
+import org.apache.pig.data.*;
+
+/**
+ * This class will exercise the nonenown data type.
+ * 
+ * @author gates
+ */
+public class TestDataCharArrayNone extends junit.framework.TestCase
+{
+
+public void testDefaultConstructor() throws Exception
+{
+	DataCharArrayNone none = new DataCharArrayNone();
+
+	assertEquals("getType", Datum.DataType.CHARARRAY, none.getType());
+	assertFalse("is null", none.isNull());
+	assertEquals("getEncoding", DataCharArray.Encoding.NONE, none.getEncoding());
+
+	assertEquals("Default constructor size before", 0L, none.size());
+	assertNull("Default constructor get before", none.get());
+	String s = "hello world";
+	none.set(s.getBytes());
+	assertEquals("Default constructor size after", 11L, none.size());
+	String n = new String(none.get());
+	assertEquals("Default constructor get after", "hello world", n);
+}
+
+public void testByteConstructor() throws Exception
+{
+	String s = "hello world";
+	DataCharArrayNone none = new DataCharArrayNone(s.getBytes());
+	assertEquals("Byte constructor size before", 11L, none.size());
+	String n = new String(none.get());
+	assertEquals("Byte constructor get before", "hello world", n);
+
+	s = "goodbye world";
+	none.set(s.getBytes());
+	assertEquals("Byte constructor after size", 13L, none.size());
+	n = new String(none.get());
+	assertEquals("Byte constructor after get", "goodbye world", n);
+}
+
+public void testToString() throws Exception
+{
+	String s = "hello world";
+	DataCharArrayNone none = new DataCharArrayNone(s.getBytes());
+	assertEquals("toString", s, none.toString());
+}
+
+public void testHashCode() throws Exception
+{
+	String s = "hello world";
+	DataCharArrayNone none1 = new DataCharArrayNone(s.getBytes());
+	DataCharArrayNone none2 = new DataCharArrayNone(s.getBytes());
+	assertEquals("same data", none1.hashCode(), none2.hashCode());
+	s = "goodbye world";
+	DataCharArrayNone none3 = new DataCharArrayNone(s.getBytes());
+	assertFalse("different data", none1.hashCode() == none3.hashCode()); 
+}
+
+public void testEquals() throws Exception
+{
+	String s = "hello world";
+	DataCharArrayNone none1 = new DataCharArrayNone(s.getBytes());
+	DataCharArrayNone none2 = new DataCharArrayNone(s.getBytes());
+
+	assertFalse("different object", none1.equals(s));
+
+	assertTrue("same data", none1.equals(none2));
+
+	s = "goodbye world";
+	DataCharArrayNone none3 = new DataCharArrayNone(s.getBytes());
+	assertFalse("different data", none1.equals(none3));
+}
+
+public void testCompareTo() throws Exception
+{
+	String s = "hello world";
+	DataCharArrayNone none1 = new DataCharArrayNone(s.getBytes());
+	DataCharArrayNone none2 = new DataCharArrayNone(s.getBytes());
+
+	assertEquals("different object less than", -1, none1.compareTo(s));
+
+	DataBag bag = new DataBag(Datum.DataType.INT);
+	assertTrue("greater than bag", none1.compareTo(bag) > 0);
+	Tuple t = new Tuple();
+	assertTrue("greater than tuple", none1.compareTo(t) > 0);
+	DataMap map = new DataMap();
+	assertTrue("greater than map", none1.compareTo(map) > 0);
+	DataInteger i = new DataInteger();
+	assertTrue("greater than integer", none1.compareTo(i) > 0);
+	DataLong l = new DataLong();
+	assertTrue("greater than long", none1.compareTo(l) > 0);
+	DataFloat f = new DataFloat();
+	assertTrue("greater than float", none1.compareTo(f) > 0);
+	DataDouble d = new DataDouble();
+	assertTrue("greater than double", none1.compareTo(d) > 0);
+	DataUnknown unk = new DataUnknown();
+	assertTrue("less than unknown", none1.compareTo(unk) < 0);
+	DataCharArrayUtf16 utf16 = new DataCharArrayUtf16(s);
+	assertTrue("greater than utf16", none1.compareTo(utf16) > 0);
+
+	assertEquals("same data equal", 0,  none1.compareTo(none2));
+
+	s = "hello dollie";
+	none2 = new DataCharArrayNone(s.getBytes());
+	assertEquals("greater than unknown with greater lexigraphic value", 1, none1.compareTo(none2));
+
+	s = "hello zylophone";
+	none2 = new DataCharArrayNone(s.getBytes());
+	assertEquals("less than unknown with lower lexigraphic value", -1, none1.compareTo(none2));
+
+	s = "hello world, goodbye moon";
+	none2 = new DataCharArrayNone(s.getBytes());
+	assertEquals("less than longer unknown", -1, none1.compareTo(none2));
+
+	s = "hello worl";
+	none2 = new DataCharArrayNone(s.getBytes());
+	assertEquals("greater than shorter unknown", 1, none1.compareTo(none2));
+}
+
+public void testWriteRead() throws Exception
+{
+	String s = "hello world";
+	DataCharArrayNone before = new DataCharArrayNone(s.getBytes());
+	File file = null;
+	file = File.createTempFile("DataCharArrayNone", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataCharArrayNone", a instanceof DataCharArrayNone);
+
+	DataCharArrayNone after = (DataCharArrayNone)a;
+		
+	byte[] beforeBytes = before.get();
+	byte[] afterBytes = after.get();
+
+	assertEquals("length", beforeBytes.length, afterBytes.length);
+	boolean same = true;
+	for (int i = 0; i < beforeBytes.length; i++) {
+		same &= beforeBytes[i] == afterBytes[i];
+	}
+	assertTrue("byte values", same);
+	file.delete();
+}
+
+}
+
+
+ 

Added: incubator/pig/branches/types/test/org/apache/pig/test/TestDataCharArrayUtf16.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestDataCharArrayUtf16.java?rev=591143&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestDataCharArrayUtf16.java (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestDataCharArrayUtf16.java Thu Nov  1 13:48:16 2007
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileInputStream;
+
+import org.junit.Test;
+
+import org.apache.pig.data.*;
+
+/**
+ * This class will exercise the utf16nown data type.
+ * 
+ * @author gates
+ */
+public class TestDataCharArrayUtf16 extends junit.framework.TestCase
+{
+
+public void testDefaultConstructor() throws Exception
+{
+	DataCharArrayUtf16 utf16 = new DataCharArrayUtf16();
+
+	assertEquals("getType", Datum.DataType.CHARARRAY, utf16.getType());
+	assertFalse("is null", utf16.isNull());
+	assertEquals("getEncoding", DataCharArray.Encoding.UTF16, utf16.getEncoding());
+
+	assertEquals("Default constructor size before", 0L, utf16.size());
+	assertNull("Default constructor get before", utf16.get());
+	String s = "hello world";
+	utf16.set(s);
+	assertEquals("Default constructor size after", 11L, utf16.size());
+	String n = utf16.get();
+	assertEquals("Default constructor get after", "hello world", n);
+}
+
+public void testByteConstructor() throws Exception
+{
+	String s = "hello world";
+	DataCharArrayUtf16 utf16 = new DataCharArrayUtf16(s.getBytes());
+	assertEquals("Byte constructor size before", 11L, utf16.size());
+	String n = utf16.get();
+	assertEquals("Byte constructor get before", "hello world", n);
+
+	s = "goodbye world";
+	utf16.set(s);
+	assertEquals("Byte constructor after size", 13L, utf16.size());
+	n = utf16.get();
+	assertEquals("Byte constructor after get", "goodbye world", n);
+}
+
+public void testStringConstructor() throws Exception
+{
+	String s = "hello world";
+	DataCharArrayUtf16 utf16 = new DataCharArrayUtf16(s);
+	assertEquals("String constructor size before", 11L, utf16.size());
+	String n = utf16.get();
+	assertEquals("String constructor get before", "hello world", n);
+}
+
+public void testToString() throws Exception
+{
+	String s = "hello world";
+	DataCharArrayUtf16 utf16 = new DataCharArrayUtf16(s);
+	assertEquals("toString", s, utf16.toString());
+}
+
+public void testHashCode() throws Exception
+{
+	String s = "hello world";
+	DataCharArrayUtf16 utf161 = new DataCharArrayUtf16(s);
+	DataCharArrayUtf16 utf162 = new DataCharArrayUtf16(s);
+	assertEquals("same data", utf161.hashCode(), utf162.hashCode());
+	s = "goodbye world";
+	DataCharArrayUtf16 utf163 = new DataCharArrayUtf16(s);
+	assertFalse("different data", utf161.hashCode() == utf163.hashCode()); 
+}
+
+public void testEquals() throws Exception
+{
+	DataCharArrayUtf16 utf16_1 = new DataCharArrayUtf16("hello world");
+	DataCharArrayUtf16 utf16_2 = new DataCharArrayUtf16("hello world");
+
+	String s = new String("hello world");
+
+	assertFalse("different object", utf16_1.equals(s));
+
+	assertTrue("same data", utf16_1.equals(utf16_2));
+
+	DataCharArrayUtf16 utf16_3 = new DataCharArrayUtf16("goodbye world");
+	assertFalse("different data", utf16_1.equals(utf16_3));
+}
+
+public void testCompareTo() throws Exception
+{
+	String s = "hello world";
+	DataCharArrayUtf16 utf16_1 = new DataCharArrayUtf16(s);
+	DataCharArrayUtf16 utf16_2 = new DataCharArrayUtf16(s);
+
+	assertEquals("different object less than", -1, utf16_1.compareTo(s));
+
+	DataBag bag = new DataBag(Datum.DataType.INT);
+	assertTrue("greater than bag", utf16_1.compareTo(bag) > 0);
+	Tuple t = new Tuple();
+	assertTrue("greater than tuple", utf16_1.compareTo(t) > 0);
+	DataMap map = new DataMap();
+	assertTrue("greater than map", utf16_1.compareTo(map) > 0);
+	DataInteger i = new DataInteger();
+	assertTrue("greater than integer", utf16_1.compareTo(i) > 0);
+	DataLong l = new DataLong();
+	assertTrue("greater than long", utf16_1.compareTo(l) > 0);
+	DataFloat f = new DataFloat();
+	assertTrue("greater than float", utf16_1.compareTo(f) > 0);
+	DataDouble d = new DataDouble();
+	assertTrue("greater than double", utf16_1.compareTo(d) > 0);
+	DataUnknown unk = new DataUnknown();
+	assertTrue("less than unknown", utf16_1.compareTo(unk) < 0);
+	DataCharArrayNone none = new DataCharArrayNone(s.getBytes());
+	assertTrue("less than none", utf16_1.compareTo(none) < 0);
+
+	assertEquals("same data equal", 0,  utf16_1.compareTo(utf16_2));
+
+	s = "hello dollie";
+	utf16_2 = new DataCharArrayUtf16(s);
+	assertTrue("greater than unknown with greater lexigraphic value", utf16_1.compareTo(utf16_2) > 0);
+
+	s = "hello zylophone";
+	utf16_2 = new DataCharArrayUtf16(s);
+	assertTrue("less than unknown with lower lexigraphic value", utf16_1.compareTo(utf16_2) < 0);
+
+	s = "hello world, goodbye moon";
+	utf16_2 = new DataCharArrayUtf16(s);
+	assertTrue("less than longer unknown", utf16_1.compareTo(utf16_2) < 0);
+
+	s = "hello worl";
+	utf16_2 = new DataCharArrayUtf16(s);
+	assertTrue("greater than shorter unknown", utf16_1.compareTo(utf16_2) > 0);
+}
+
+
+public void testWriteRead() throws Exception
+{
+	String s = "hello world";
+	DataCharArrayUtf16 before = new DataCharArrayUtf16(s);
+	File file = null;
+	file = File.createTempFile("DataCharArrayUtf16", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataCharArrayUtf16", a instanceof DataCharArrayUtf16);
+
+	DataCharArrayUtf16 after = (DataCharArrayUtf16)a;
+		
+	assertEquals("after read/write", before.get(), after.get());
+
+	file.delete();
+}
+
+}
+
+
+ 

Added: incubator/pig/branches/types/test/org/apache/pig/test/TestDataDouble.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestDataDouble.java?rev=591143&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestDataDouble.java (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestDataDouble.java Thu Nov  1 13:48:16 2007
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * fstributed with this work for adftional information
+ * regarfng 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
+ * fstributed under the License is fstributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileInputStream;
+
+import org.junit.Test;
+
+import org.apache.pig.data.*;
+
+/**
+ * This class will exercise the fnown data type.
+ * 
+ * @author gates
+ */
+public class TestDataDouble extends junit.framework.TestCase
+{
+
+public void testDefaultConstructor() throws Exception
+{
+	DataDouble d = new DataDouble();
+
+	assertEquals("getType", Datum.DataType.DOUBLE, d.getType());
+	assertFalse("is null", d.isNull());
+
+	assertEquals("Default constructor get before", 0.0, d.get());
+	d.set(0.1);
+	assertEquals("Default constructor get after", 0.1, d.get());
+}
+
+public void testValueConstructor() throws Exception
+{
+	DataDouble d = new DataDouble(2.99792458e108);
+	assertEquals("Byte constructor get before", 2.99792458e108, d.get());
+
+	d.set(1.60217653e-149);
+	assertEquals("Byte constructor after get", 1.60217653e-149, d.get());
+}
+
+public void testToString() throws Exception
+{
+	DataDouble d = new DataDouble(-99.3234);
+	assertEquals("toString", "-99.3234", d.toString());
+}
+
+public void testHashCode() throws Exception
+{
+	String s = "hello world";
+	DataDouble d1 = new DataDouble(8.2);
+	DataDouble d2 = new DataDouble(8.2);
+	assertEquals("same data", d1.hashCode(), d2.hashCode());
+	s = "goodbye world";
+	DataDouble f3 = new DataDouble(9.3);
+	assertFalse("ffferent data", d1.hashCode() == f3.hashCode()); 
+}
+
+public void testEquals() throws Exception
+{
+	DataDouble d1 = new DataDouble(-1.2393);
+	DataDouble d2 = new DataDouble(-1.2393);
+
+	Double dd = new Double(-1.2393);
+
+	assertFalse("different object", d1.equals(dd));
+
+	assertTrue("same data", d1.equals(d2));
+
+	DataDouble d3 = new DataDouble(37.2322e39);
+	assertFalse("different data", d1.equals(d3));
+}
+
+public void testCompareTo() throws Exception
+{
+	DataDouble d1 = new DataDouble(3.1415);
+	DataDouble d2 = new DataDouble(3.1415);
+
+	assertEquals("different object less than", -1, d1.compareTo(new Integer(3)));
+
+	DataBag bag = new DataBag(Datum.DataType.INT);
+	assertTrue("greater than bag", d1.compareTo(bag) > 0);
+	Tuple t = new Tuple();
+	assertTrue("greater than tuple", d1.compareTo(t) > 0);
+	DataMap map = new DataMap();
+	assertTrue("greater than map", d1.compareTo(map) > 0);
+	DataInteger i = new DataInteger(3);
+	assertTrue("greater than integer", d1.compareTo(i) > 0);
+	DataLong dl = new DataLong();
+	assertTrue("greater than long", d1.compareTo(dl) > 0);
+	DataFloat f = new DataFloat(3.1415f);
+	assertTrue("greater than float", d1.compareTo(f) > 0);
+	DataUnknown unk = new DataUnknown();
+	assertTrue("less than unknown", d1.compareTo(unk) < 0);
+	DataCharArrayUtf16 utf16 = new DataCharArrayUtf16("hello world");
+	assertTrue("less than utf16", d1.compareTo(utf16) < 0);
+
+	assertEquals("same data equal", 0,  d1.compareTo(d2));
+
+	d2 = new DataDouble(4.0);
+	assertEquals("less than unknown with greater value", -1, d1.compareTo(d2));
+
+	d2 = new DataDouble(3.0);
+	assertEquals("greater than unknown with lower value", 1, d1.compareTo(d2));
+}
+
+public void testWriteRead() throws Exception
+{
+	DataDouble before = new DataDouble(17.9);
+	File file = null;
+	file = File.createTempFile("DataDouble", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataDouble", a instanceof DataDouble);
+
+	DataDouble after = (DataDouble)a;
+		
+	assertEquals("after read/write", before.get(), after.get());
+	file.delete();
+}
+
+}
+
+
+ 

Added: incubator/pig/branches/types/test/org/apache/pig/test/TestDataFloat.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestDataFloat.java?rev=591143&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestDataFloat.java (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestDataFloat.java Thu Nov  1 13:48:16 2007
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * fstributed with this work for adftional information
+ * regarfng 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
+ * fstributed under the License is fstributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileInputStream;
+
+import org.junit.Test;
+
+import org.apache.pig.data.*;
+
+/**
+ * This class will exercise the fnown data type.
+ * 
+ * @author gates
+ */
+public class TestDataFloat extends junit.framework.TestCase
+{
+
+public void testDefaultConstructor() throws Exception
+{
+	DataFloat f = new DataFloat();
+
+	assertEquals("getType", Datum.DataType.FLOAT, f.getType());
+	assertFalse("is null", f.isNull());
+
+	assertEquals("Default constructor get before", 0.0f, f.get());
+	f.set(0.1f);
+	assertEquals("Default constructor get after", 0.1f, f.get());
+}
+
+public void testValueConstructor() throws Exception
+{
+	DataFloat f = new DataFloat(2.99792458e8f);
+	assertEquals("Byte constructor get before", 2.99792458e8f, f.get());
+
+	f.set(1.60217653e-14f);
+	assertEquals("Byte constructor after get", 1.60217653e-14f, f.get());
+}
+
+public void testToString() throws Exception
+{
+	DataFloat f = new DataFloat(-99.3234f);
+	assertEquals("toString", "-99.3234", f.toString());
+}
+
+public void testHashCode() throws Exception
+{
+	String s = "hello world";
+	DataFloat f1 = new DataFloat(8.2f);
+	DataFloat f2 = new DataFloat(8.2f);
+	assertEquals("same data", f1.hashCode(), f2.hashCode());
+	s = "goodbye world";
+	DataFloat f3 = new DataFloat(9.3f);
+	assertFalse("ffferent data", f1.hashCode() == f3.hashCode()); 
+}
+
+public void testEquals() throws Exception
+{
+	DataFloat f1 = new DataFloat(-1.0f);
+	DataFloat f2 = new DataFloat(-1.0f);
+
+	Float ff = new Float(-1.0f);
+
+	assertFalse("different object", f1.equals(ff));
+
+	assertTrue("same data", f1.equals(f2));
+
+	DataFloat f3 = new DataFloat(37.393f);
+	assertFalse("different data", f1.equals(f3));
+}
+
+public void testCompareTo() throws Exception
+{
+	DataFloat f1 = new DataFloat(3.1415f);
+	DataFloat f2 = new DataFloat(3.1415f);
+
+	assertEquals("different object less than", -1, f1.compareTo(new Integer(3)));
+
+	DataBag bag = new DataBag(Datum.DataType.INT);
+	assertTrue("greater than bag", f1.compareTo(bag) > 0);
+	Tuple t = new Tuple();
+	assertTrue("greater than tuple", f1.compareTo(t) > 0);
+	DataMap map = new DataMap();
+	assertTrue("greater than map", f1.compareTo(map) > 0);
+	DataInteger i = new DataInteger(3);
+	assertTrue("greater than integer", f1.compareTo(i) > 0);
+	DataLong dl = new DataLong();
+	assertTrue("greater than long", f1.compareTo(dl) > 0);
+	DataDouble d = new DataDouble(3.1415);
+	assertTrue("less than double", f1.compareTo(d) < 0);
+	DataUnknown unk = new DataUnknown();
+	assertTrue("less than unknown", f1.compareTo(unk) < 0);
+	DataCharArrayUtf16 utf16 = new DataCharArrayUtf16("hello world");
+	assertTrue("less than utf16", f1.compareTo(utf16) < 0);
+
+	assertEquals("same data equal", 0,  f1.compareTo(f2));
+
+	f2 = new DataFloat(4.0f);
+	assertEquals("less than unknown with greater value", -1, f1.compareTo(f2));
+
+	f2 = new DataFloat(3.0f);
+	assertEquals("greater than unknown with lower value", 1, f1.compareTo(f2));
+}
+
+public void testWriteRead() throws Exception
+{
+	DataFloat before = new DataFloat(17.9f);
+	File file = null;
+	file = File.createTempFile("DataFloat", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataFloat", a instanceof DataFloat);
+
+	DataFloat after = (DataFloat)a;
+		
+	assertEquals("after read/write", before.get(), after.get());
+	file.delete();
+}
+
+}
+
+
+ 

Added: incubator/pig/branches/types/test/org/apache/pig/test/TestDataInteger.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestDataInteger.java?rev=591143&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestDataInteger.java (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestDataInteger.java Thu Nov  1 13:48:16 2007
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileInputStream;
+
+import org.junit.Test;
+
+import org.apache.pig.data.*;
+
+/**
+ * This class will exercise the dinown data type.
+ * 
+ * @author gates
+ */
+public class TestDataInteger extends junit.framework.TestCase
+{
+
+public void testDefaultConstructor() throws Exception
+{
+	DataInteger di = new DataInteger();
+
+	assertEquals("getType", Datum.DataType.INT, di.getType());
+	assertFalse("is null", di.isNull());
+
+	assertEquals("Default constructor get before", 0, di.get());
+	di.set(11);
+	assertEquals("Default constructor get after", 11, di.get());
+}
+
+public void testValueConstructor() throws Exception
+{
+	DataInteger di = new DataInteger(3231132);
+	assertEquals("Byte constructor get before", 3231132, di.get());
+
+	di.set(-1);
+	assertEquals("Byte constructor after get", -1, di.get());
+}
+
+public void testToString() throws Exception
+{
+	DataInteger di = new DataInteger(99);
+	assertEquals("toString", "99", di.toString());
+}
+
+public void testHashCode() throws Exception
+{
+	DataInteger di1 = new DataInteger(8);
+	DataInteger di2 = new DataInteger(8);
+	assertEquals("same data", di1.hashCode(), di2.hashCode());
+	DataInteger di3 = new DataInteger(9);
+	assertFalse("different data", di1.hashCode() == di3.hashCode()); 
+}
+
+public void testEquals() throws Exception
+{
+	DataInteger di1 = new DataInteger(-1);
+	DataInteger di2 = new DataInteger(-1);
+
+	Integer ii = new Integer(-1);
+
+	assertFalse("different object", di1.equals(ii));
+
+	assertTrue("same data", di1.equals(di2));
+
+	DataInteger di3 = new DataInteger(37);
+	assertFalse("different data", di1.equals(di3));
+}
+
+public void testCompareTo() throws Exception
+{
+	DataInteger di1 = new DataInteger(3);
+	DataInteger di2 = new DataInteger(3);
+
+	assertEquals("different object less than", -1, di1.compareTo(new Integer(3)));
+
+	DataBag bag = new DataBag(Datum.DataType.INT);
+	assertTrue("greater than bag", di1.compareTo(bag) > 0);
+	Tuple t = new Tuple();
+	assertTrue("greater than tuple", di1.compareTo(t) > 0);
+	DataMap map = new DataMap();
+	assertTrue("greater than map", di1.compareTo(map) > 0);
+	DataLong l = new DataLong(3);
+	assertTrue("less than long", di1.compareTo(l) < 0);
+	DataFloat f = new DataFloat();
+	assertTrue("less than float", di1.compareTo(f) < 0);
+	DataDouble d = new DataDouble();
+	assertTrue("less than double", di1.compareTo(d) < 0);
+	DataUnknown unk = new DataUnknown();
+	assertTrue("less than unknown", di1.compareTo(unk) < 0);
+	DataCharArrayUtf16 utf16 = new DataCharArrayUtf16("hello world");
+	assertTrue("less than none", di1.compareTo(utf16) < 0);
+
+	assertEquals("same data equal", 0,  di1.compareTo(di2));
+
+	di2 = new DataInteger(5);
+	assertEquals("less than int with greater value", -1, di1.compareTo(di2));
+
+	di2 = new DataInteger(1);
+	assertEquals("greater than int with lower value", 1, di1.compareTo(di2));
+}
+
+
+public void testWriteRead() throws Exception
+{
+	DataInteger before = new DataInteger(17);
+	File file = null;
+	file = File.createTempFile("DataInteger", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataInteger", a instanceof DataInteger);
+
+	DataInteger after = (DataInteger)a;
+		
+	assertEquals("after read/write", before.get(), after.get());
+	file.delete();
+}
+
+}
+
+
+ 

Added: incubator/pig/branches/types/test/org/apache/pig/test/TestDataLong.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestDataLong.java?rev=591143&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestDataLong.java (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestDataLong.java Thu Nov  1 13:48:16 2007
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * dlstributed with this work for addltional information
+ * regardlng 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
+ * dlstributed under the License is dlstributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileInputStream;
+
+import org.junit.Test;
+
+import org.apache.pig.data.*;
+
+/**
+ * This class will exercise the dlnown data type.
+ * 
+ * @author gates
+ */
+public class TestDataLong extends junit.framework.TestCase
+{
+
+public void testDefaultConstructor() throws Exception
+{
+	DataLong dl = new DataLong();
+
+	assertEquals("getType", Datum.DataType.LONG, dl.getType());
+	assertFalse("is null", dl.isNull());
+
+	assertEquals("Default constructor get before", 0L, dl.get());
+	dl.set(11L);
+	assertEquals("Default constructor get after", 11L, dl.get());
+}
+
+public void testValueConstructor() throws Exception
+{
+	DataLong dl = new DataLong(323113232322L);
+	assertEquals("Byte constructor get before", 323113232322L, dl.get());
+
+	dl.set(-1L);
+	assertEquals("Byte constructor after get", -1L, dl.get());
+}
+
+public void testToString() throws Exception
+{
+	DataLong dl = new DataLong(-323113232322L);
+	assertEquals("toString", "-323113232322", dl.toString());
+}
+
+public void testHashCode() throws Exception
+{
+	String s = "hello world";
+	DataLong dl1 = new DataLong(8L);
+	DataLong dl2 = new DataLong(8L);
+	assertEquals("same data", dl1.hashCode(), dl2.hashCode());
+	s = "goodbye world";
+	DataLong dl3 = new DataLong(9L);
+	assertFalse("dlfferent data", dl1.hashCode() == dl3.hashCode()); 
+}
+
+public void testEquals() throws Exception
+{
+	DataLong dl1 = new DataLong(-1L);
+	DataLong dl2 = new DataLong(-1L);
+
+	Long ll = new Long(-1L);
+
+	assertFalse("different object", dl1.equals(ll));
+
+	assertTrue("same data", dl1.equals(dl2));
+
+	DataLong dl3 = new DataLong(37L);
+	assertFalse("different data", dl1.equals(dl3));
+}
+
+public void testCompareTo() throws Exception
+{
+	DataLong dl1 = new DataLong(3);
+	DataLong dl2 = new DataLong(3);
+
+	assertEquals("different object less than", -1, dl1.compareTo(new Integer(3)));
+
+	DataBag bag = new DataBag(Datum.DataType.INT);
+	assertTrue("greater than bag", dl1.compareTo(bag) > 0);
+	Tuple t = new Tuple();
+	assertTrue("greater than tuple", dl1.compareTo(t) > 0);
+	DataMap map = new DataMap();
+	assertTrue("greater than map", dl1.compareTo(map) > 0);
+	DataInteger i = new DataInteger(3);
+	assertTrue("greater than integer", dl1.compareTo(i) > 0);
+	DataFloat f = new DataFloat();
+	assertTrue("less than float", dl1.compareTo(f) < 0);
+	DataDouble d = new DataDouble();
+	assertTrue("less than double", dl1.compareTo(d) < 0);
+	DataUnknown unk = new DataUnknown();
+	assertTrue("less than unknown", dl1.compareTo(unk) < 0);
+	DataCharArrayUtf16 utf16 = new DataCharArrayUtf16("hello world");
+	assertTrue("less than utf16", dl1.compareTo(utf16) < 0);
+
+	assertEquals("same data equal", 0,  dl1.compareTo(dl2));
+
+	dl2 = new DataLong(5);
+	assertEquals("less than unknown with greater value", -1, dl1.compareTo(dl2));
+
+	dl2 = new DataLong(1);
+	assertEquals("greater than unknown with lower value", 1, dl1.compareTo(dl2));
+}
+
+public void testWriteRead() throws Exception
+{
+	DataLong before = new DataLong(17L);
+	File file = null;
+	file = File.createTempFile("DataLong", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataLong", a instanceof DataLong);
+
+	DataLong after = (DataLong)a;
+		
+	assertEquals("after read/write", before.get(), after.get());
+	file.delete();
+}
+
+}
+
+
+ 

Added: incubator/pig/branches/types/test/org/apache/pig/test/TestDataMap.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestDataMap.java?rev=591143&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestDataMap.java (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestDataMap.java Thu Nov  1 13:48:16 2007
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.pig.data.*;
+
+/**
+ * This class will exercise the data map data type.
+ * 
+ * @author gates
+ */
+public class TestDataMap extends junit.framework.TestCase
+{
+
+public void testDefaultConstructor() throws Exception
+{
+	DataMap map = new DataMap();
+
+	assertEquals("getType", Datum.DataType.MAP, map.getType());
+	assertFalse("is null", map.isNull());
+
+	assertEquals("Default constructor size before", 0, map.size());
+	String s = "hello world";
+	DataUnknown key = new DataUnknown(s.getBytes());
+	Datum d = map.get(key);
+	assertTrue("get with no entries in map", d.isNull());
+	DataInteger val = new DataInteger(42);
+
+	map.put(key, val);
+	assertEquals("Default constructor size after", 1, map.size());
+	d = map.get(key);
+	assertNotNull("get with entry in map", d);
+	assertTrue("val isa integer", d instanceof DataInteger);
+	assertEquals("value of val", 42, ((DataInteger)d).get());
+}
+
+public void testBigMap() throws Exception
+{
+	DataMap map = new DataMap();
+
+	for (int i = 0; i < 10000; i++) {
+		Integer ii = new Integer(i);
+		DataCharArrayUtf16 key = new DataCharArrayUtf16(ii.toString());
+		DataInteger val = new DataInteger(i);
+		map.put(key, val);
+	}
+
+	assertEquals("big size after loading", 10000, map.size());
+	DataCharArrayUtf16 key = new DataCharArrayUtf16("no such key");
+	Datum d = map.get(key);
+	assertTrue("get no such key", d.isNull());
+
+	for (int i = 9999; i >= 0; i--) {
+		Integer ii = new Integer(i);
+		key = new DataCharArrayUtf16(ii.toString());
+		Datum val = map.get(key);
+		assertFalse("val should not be null", val.isNull());
+		assertTrue("val isa integer", val instanceof DataInteger);
+		assertEquals("value of val", i, ((DataInteger)val).get());
+	}
+}
+
+public void testToString() throws Exception
+{
+	DataMap map = new DataMap();
+
+	DataCharArrayUtf16 key = new DataCharArrayUtf16("aaa");
+	Datum val = new DataLong(1L);
+	map.put(key, val);
+	key = new DataCharArrayUtf16("bbb");
+	val = new DataFloat(2.0f);
+	map.put(key, val);
+	key = new DataCharArrayUtf16("ccc");
+	val = new DataDouble(3.14159);
+	map.put(key, val);
+
+	assertEquals("toString", "[aaa#1, ccc#3.14159, bbb#2.0]", map.toString());
+}
+
+public void testHashCode() throws Exception
+{
+	DataMap map1 = new DataMap();
+	DataCharArrayUtf16 key1 = new DataCharArrayUtf16("aaa");
+	DataLong val1 = new DataLong(1L);
+	map1.put(key1, val1);
+
+	DataMap map2 = new DataMap();
+	DataCharArrayUtf16 key2 = new DataCharArrayUtf16("aaa");
+	DataLong val2 = new DataLong(1L);
+	map2.put(key2, val2);
+
+	assertEquals("same data", map1.hashCode(), map2.hashCode());
+
+	DataMap map3 = new DataMap();
+	DataCharArrayUtf16 key3 = new DataCharArrayUtf16("aaa");
+	DataLong val3 = new DataLong(2L);
+	map3.put(key3, val3);
+
+	assertFalse("different data", map1.hashCode() == map3.hashCode()); 
+}
+
+public void testEquals() throws Exception
+{
+	DataMap map1 = new DataMap();
+	DataMap map2 = new DataMap();
+
+	map1.put(new DataInteger(3), new DataInteger(4));
+	map2.put(new DataInteger(3), new DataInteger(4));
+
+	assertFalse("different object", map1.equals(new String()));
+
+	assertTrue("same data", map1.equals(map2));
+
+	map2 = new DataMap();
+	map2.put(new DataInteger(3), new DataInteger(5));
+	assertFalse("different data", map1.equals(map2));
+
+	map2 = new DataMap();
+	map2.put(new DataInteger(3), new DataInteger(4));
+	map2.put(new DataInteger(4), new DataInteger(4));
+	assertFalse("different size", map1.equals(map2));
+}
+
+public void testCompareTo() throws Exception
+{
+	DataMap map1 = new DataMap();
+	DataMap map2 = new DataMap();
+
+	map1.put(new DataInteger(3), new DataInteger(4));
+	map2.put(new DataInteger(3), new DataInteger(4));
+
+	assertEquals("different object less than", -1, map1.compareTo(new String()));
+
+	DataBag bag = new DataBag(Datum.DataType.INT);
+	assertTrue("greater than bag", map1.compareTo(bag) > 0);
+	Tuple t = new Tuple();
+	assertTrue("greater than tuple", map1.compareTo(t) > 0);
+	DataInteger i = new DataInteger();
+	assertTrue("less than integer", map1.compareTo(i) < 0);
+	DataLong l = new DataLong();
+	assertTrue("less than long", map1.compareTo(l) < 0);
+	DataFloat f = new DataFloat();
+	assertTrue("less than float", map1.compareTo(f) < 0);
+	DataDouble d = new DataDouble();
+	assertTrue("less than double", map1.compareTo(d) < 0);
+	DataUnknown unk = new DataUnknown();
+	assertTrue("less than unknown", map1.compareTo(unk) < 0);
+	DataCharArrayUtf16 utf16 = new DataCharArrayUtf16();
+	assertTrue("less than utf16", map1.compareTo(utf16) < 0);
+
+	assertEquals("same data equal", 0, map1.compareTo(map2));
+
+	map2 = new DataMap();
+	map2.put(new DataInteger(2), new DataInteger(4));
+	assertTrue("greater than map with lesser key", map1.compareTo(map2) > 0);
+
+	map2 = new DataMap();
+	map2.put(new DataInteger(3), new DataInteger(3));
+	assertTrue("greater than map with lesser value", map1.compareTo(map2) > 0);
+
+	map2 = new DataMap();
+	map2.put(new DataInteger(4), new DataInteger(4));
+	assertTrue("less than map with greater key", map1.compareTo(map2) < 0);
+
+	map2 = new DataMap();
+	map2.put(new DataInteger(3), new DataInteger(5));
+	assertTrue("less than map with greater value", map1.compareTo(map2) < 0);
+
+	map2 = new DataMap();
+	map2.put(new DataInteger(3), new DataInteger(5));
+	map2.put(new DataInteger(4), new DataInteger(5));
+	assertTrue("less than bigger map", map1.compareTo(map2) < 0);
+
+	map2 = new DataMap();
+	assertTrue("greater than smaller map", map1.compareTo(map2) > 0);
+}
+
+public void testWriteRead() throws Exception
+{
+	DataMap before = new DataMap();
+
+	String s = new String("aaa");
+	DataUnknown keyUnknown = new DataUnknown(s.getBytes());
+	s = new String("zzz");
+	Datum val = new DataUnknown(s.getBytes());
+	before.put(keyUnknown, val);
+
+	DataInteger keyInt = new DataInteger(1);
+	val = new DataInteger(99);
+	before.put(keyInt, val);
+
+	DataLong keyLong = new DataLong(1L);
+	val = new DataLong(99000000000L);
+	before.put(keyLong, val);
+
+	DataFloat keyFloat = new DataFloat(1.0f);
+	val = new DataFloat(3.2e32f);
+	before.put(keyFloat, val);
+
+	DataDouble keyDouble = new DataDouble(1.0);
+	val = new DataDouble(3.2e132);
+	before.put(keyDouble, val);
+
+	DataCharArrayUtf16 keyUtf16 = new DataCharArrayUtf16("aaa");
+	val = new DataCharArrayUtf16("yyy");
+	before.put(keyUtf16, val);
+
+	s = new String("aaa");
+	DataCharArrayNone keyNone = new DataCharArrayNone(s.getBytes());
+	s = new String("xxx");
+	val = new DataCharArrayNone(s.getBytes());
+	before.put(keyNone, val);
+
+	File file = null;
+	file = File.createTempFile("DataMapUnknown", "put");
+	FileOutputStream fos = new FileOutputStream(file);
+	DataOutput out = new DataOutputStream(fos);
+	before.write(out);
+	fos.close();
+
+	FileInputStream fis = new FileInputStream(file);
+	DataInput in = new DataInputStream(fis);
+	Datum a = DatumImpl.readDatum(in);
+
+	assertTrue("isa DataMap", a instanceof DataMap);
+
+	DataMap after = (DataMap)a;
+
+	assertEquals("after read, size", 7, after.size()); 
+
+	s = new String("no such key");
+	DataUnknown nosuch = new DataUnknown(s.getBytes());
+	Datum d = after.get(nosuch);
+	assertTrue("after read, no such key", d.isNull());
+
+	Datum valAfter = after.get(keyUnknown);
+	assertTrue("valAfter isa integer", valAfter instanceof DataUnknown);
+	for (int i = 0; i < 3; i++) {
+		assertEquals("value of valAfter", (byte)0x7a,
+			((DataUnknown)valAfter).get()[i]);
+	}
+
+	valAfter = after.get(keyInt);
+	assertTrue("valAfter isa integer", valAfter instanceof DataInteger);
+	assertEquals("value of valAfter", 99, ((DataInteger)valAfter).get());
+
+	valAfter = after.get(keyLong);
+	assertTrue("valAfter isa long", valAfter instanceof DataLong);
+	assertEquals("value of valAfter", 99000000000L, ((DataLong)valAfter).get());
+
+	valAfter = after.get(keyFloat);
+	assertTrue("valAfter isa float", valAfter instanceof DataFloat);
+	assertEquals("value of valAfter", 3.2e32f, ((DataFloat)valAfter).get());
+
+	valAfter = after.get(keyDouble);
+	assertTrue("valAfter isa double", valAfter instanceof DataDouble);
+	assertEquals("value of valAfter", 3.2e132, ((DataDouble)valAfter).get());
+
+	valAfter = after.get(keyUtf16);
+	assertTrue("valAfter isa utf16", valAfter instanceof DataCharArrayUtf16);
+	assertEquals("value of valAfter", "yyy", ((DataCharArrayUtf16)valAfter).get());
+
+	valAfter = after.get(keyNone);
+	assertTrue("valAfter isa none", valAfter instanceof DataCharArrayNone);
+	for (int i = 0; i < 3; i++) {
+		assertEquals("value of valAfter", (byte)0x78,
+			((DataCharArrayNone)valAfter).get()[i]);
+	}
+
+	file.delete();
+}
+
+}
+
+
+ 

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=591143&r1=591142&r2=591143&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 Nov  1 13:48:16 2007
@@ -73,6 +73,7 @@
         assertFalse(da1.compareTo(da2) > 0);
     }
 
+	/* Replaced by TestTuple.java
     @Test
     public void testTuple() throws Exception {
         int arity = 5;
@@ -159,7 +160,9 @@
         n1.appendTuple(n2);
         assertTrue(n1.arity() == n1Arity + n2Arity);
     }
+	*/
 
+	/* Replaced by TestDataBag.java 
     @Test
     public void testDataBag() throws Exception {
         int[] input1 = { 1, 2, 3, 4, 5 };
@@ -195,6 +198,7 @@
         }   
         assertTrue(caught);
     }
+	*/
 
     @Test
     
@@ -211,10 +215,11 @@
     private void testBigDataBag(long freeMemoryToMaintain, int numItems) throws Exception {
     	BigDataBag.FREE_MEMORY_TO_MAINTAIN = freeMemoryToMaintain;
     	File tmp = File.createTempFile("test", "bag").getParentFile();
-        BigDataBag bag = new BigDataBag(tmp);
-        Iterator<Tuple> it;
+        BigDataBag bag = new BigDataBag(Datum.DataType.TUPLE, tmp);
+        Iterator<Datum> it;
         int count;
-        String last;
+        //String last;
+		Tuple lastT = null;
     
         Random r = new Random();
         
@@ -237,7 +242,7 @@
         it = bag.content();
         count = 0;
         while(it.hasNext()) {
-            Tuple t = it.next();
+            Tuple t = (Tuple)it.next();
             int ix = Integer.parseInt(t.getAtomField(0).strval(), 16);
             assertTrue(Integer.toString(ix).equals(t.getAtomField(1).strval()));
             assertEquals(lastI+1, ix);
@@ -262,12 +267,17 @@
         
         it = bag.content();
         count = 0;
-        last= "";
+        // last= "";
+		lastT = new Tuple();
         while(it.hasNext()) {
-            Tuple t = it.next();
+            Tuple t = (Tuple)it.next();
+			/*
             String next = t.getAtomField(0).strval();
             assertTrue(last.compareTo(next)<=0);
             last = next;
+			*/
+			assertTrue("last should be <= next", lastT.compareTo(t) <= 0);
+			lastT = t;
             count++;
         }
 
@@ -288,12 +298,17 @@
 
         it = bag.content();
         count = 0;
-        last= "";
+        //last= "";
+		lastT = new Tuple();
         while(it.hasNext()) {
-            Tuple t = it.next();
+            Tuple t = (Tuple)it.next();
+			/*
             String next = t.getAtomField(0).strval();
             assertTrue(last.compareTo(next)<=0);
             last = next;
+			*/
+			assertTrue("last should be <= next", lastT.compareTo(t) <= 0);
+			lastT = t;
             count++;
         }
 
@@ -316,12 +331,17 @@
 
         it = bag.content();
         count = 0;
-        last= "";
+        //last= "";
+		lastT = new Tuple();
         while(it.hasNext()) {
-            Tuple t = it.next();
+            Tuple t = (Tuple)it.next();
+			/*
             String next = t.getAtomField(0).strval();
             assertTrue(last.compareTo(next)<0);
             last = next;
+			*/
+			assertTrue("last should be <= next", lastT.compareTo(t) <= 0);
+			lastT = t;
             count++;
         }
 
@@ -345,12 +365,17 @@
 
         it = bag.content();
         count = 0;
-        last= "";
+        //last= "";
+		lastT = new Tuple();
         while(it.hasNext()) {
-            Tuple t = it.next();
+            Tuple t = (Tuple)it.next();
+			/*
             String next = t.getAtomField(0).strval();
             assertTrue(last.compareTo(next)<0);
             last = next;
+			*/
+			assertTrue("last should be <= next", lastT.compareTo(t) <= 0);
+			lastT = t;
             count++;
         }
 
@@ -359,12 +384,17 @@
         //Check if it gives the correct contents the second time around
         it = bag.content();
         count = 0;
-        last= "";
+        //last= "";
+		lastT = new Tuple();
         while(it.hasNext()) {
-            Tuple t = it.next();
+            Tuple t = (Tuple)it.next();
+			/*
             String next = t.getAtomField(0).strval();
             assertTrue(last.compareTo(next)<0);
             last = next;
+			*/
+			assertTrue("last should be <= next", lastT.compareTo(t) <= 0);
+			lastT = t;
             count++;
         }