Modified: 
incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java
URL: 
http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java?rev=630357&r1=630356&r2=630357&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java 
(original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestEvalPipeline.java 
Fri Feb 22 15:44:37 2008
@@ -15,298 +15,307 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.pig.test;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.StringTokenizer;
-
-import org.junit.Test;
-
-import org.apache.pig.EvalFunc;
-import org.apache.pig.PigServer;
-import org.apache.pig.builtin.BinStorage;
-import org.apache.pig.builtin.PigStorage;
-import org.apache.pig.builtin.TextLoader;
-import org.apache.pig.data.*;
-import org.apache.pig.impl.io.FileLocalizer;
-import org.apache.pig.impl.io.PigFile;
-
-import junit.framework.TestCase;
-
-public class TestEvalPipeline extends TestCase {
-       
-       String initString = "mapreduce";
-       
-       
-       static public class MyBagFunction extends EvalFunc<DataBag>{
-               @Override
-               public void exec(Tuple input, DataBag output) throws 
IOException {
-                       output.add(new Tuple("a"));
-                       output.add(new Tuple("a"));
-                       output.add(new Tuple("a"));
-                       
-               }
-       }
-       
-       
-       private File createFile(String[] data) throws Exception{
-               File f = File.createTempFile("tmp", "");
-               PrintWriter pw = new PrintWriter(f);
-               for (int i=0; i<data.length; i++){
-                       pw.println(data[i]);
-               }
-               pw.close();
-               return f;
-       }
-       
-       @Test
-       public void testFunctionInsideFunction() throws Exception{
-               PigServer pigServer = new PigServer(initString);
-               
-               File f1 = createFile(new String[]{"a:1","b:1","a:1"});
-
-               pigServer.registerQuery("a = load 'file:" + f1 + "' using " + 
PigStorage.class.getName() + "(':');");
-               pigServer.registerQuery("b = foreach a generate '1'-'1'/'1';");
-               Iterator<Tuple> iter  = pigServer.openIterator("b");
-               
-               for (int i=0 ;i<3; i++){
-                       assertEquals(iter.next().getAtomField(0).numval(), 0.0);
-               }
-               
-       }
-       
-       @Test
-       public void testJoin() throws Exception{
-               PigServer pigServer = new PigServer(initString);
-               
-               File f1 = createFile(new String[]{"a:1","b:1","a:1"});
-               File f2 = createFile(new String[]{"b","b","a"});
-               
-               pigServer.registerQuery("a = load 'file:" + f1 + "' using " + 
PigStorage.class.getName() + "(':');");
-               pigServer.registerQuery("b = load 'file:" + f2 + "';");
-               pigServer.registerQuery("c = cogroup a by $0, b by $0;");       
        
-               pigServer.registerQuery("d = foreach c generate 
flatten($1),flatten($2);");
-               
-               Iterator<Tuple> iter = pigServer.openIterator("d");
-               int count = 0;
-               while(iter.hasNext()){
-                       Tuple t = iter.next();
-                       
assertTrue(t.getAtomField(0).strval().equals(t.getAtomField(2).strval()));
-                       count++;
-               }
-               assertEquals(count, 4);
-       }
-       
-       @Test
-       public void testDriverMethod() throws Exception{
-               PigServer pigServer = new PigServer(initString);
-               File f = File.createTempFile("tmp", "");
-               PrintWriter pw = new PrintWriter(f);
-               pw.println("a");
-               pw.println("a");
-               pw.close();
-               pigServer.registerQuery("a = foreach (load 'file:" + f + "') 
generate '1', flatten(" + MyBagFunction.class.getName() + "(*));");
-               pigServer.registerQuery("b = foreach a generate $0, 
flatten($1);");
-               Iterator<Tuple> iter = pigServer.openIterator("a");
-               int count = 0;
-               while(iter.hasNext()){
-                       Tuple t = iter.next();
-                       assertTrue(t.getAtomField(0).strval().equals("1"));
-                       assertTrue(t.getAtomField(1).strval().equals("a"));
-                       count++;
-               }
-               assertEquals(count, 6);
-               f.delete();
-       }
-       
-       
-       @Test
-       public void testMapLookup() throws IOException{
-               PigServer pigServer = new PigServer(initString);
-               DataBag b = BagFactory.getInstance().newDefaultBag();
-               DataMap colors = new DataMap();
-               colors.put("apple","red");
-               colors.put("orange","orange");
-               
-               DataMap weights = new DataMap();
-               weights.put("apple","0.1");
-               weights.put("orange","0.3");
-               
-               Tuple t = new Tuple();
-               t.appendField(colors);
-               t.appendField(weights);
-               b.add(t);
-               
-               String fileName = "file:"+File.createTempFile("tmp", "");
-               PigFile f = new PigFile(fileName);
-               f.store(b, new BinStorage(), pigServer.getPigContext());
-               
-               
-               pigServer.registerQuery("a = load '" + fileName + "' using 
BinStorage();");
-               pigServer.registerQuery("b = foreach a generate 
$0#'apple',flatten($1#'orange');");
-               Iterator<Tuple> iter = pigServer.openIterator("b");
-               t = iter.next();
-               assertEquals(t.getAtomField(0).strval(), "red");
-               assertEquals(t.getAtomField(1).numval(), 0.3);
-               assertFalse(iter.hasNext());
-       }
-       
-       
-       static public class TitleNGrams extends EvalFunc<DataBag> {
-               
-               @Override
-               public void exec(Tuple input, DataBag output) throws 
IOException {      
-                   String str = input.getAtomField(0).strval();
-                       
-                       String title = str;
-
-                       if (title != null) {
-                               List<String> nGrams = makeNGrams(title);
-                               
-                               for (Iterator<String> it = nGrams.iterator(); 
it.hasNext(); ) {
-                                       Tuple t = new Tuple(1);
-                                       t.setField(0, it.next());
-                                       output.add(t);
-                               }
-                       }
-           }
-               
-               
-               List<String> makeNGrams(String str) {
-                       List<String> tokens = new ArrayList<String>();
-                       
-                       StringTokenizer st = new StringTokenizer(str);
-                       while (st.hasMoreTokens())
-                               tokens.add(st.nextToken());
-                       
-                       return nGramHelper(tokens, new ArrayList<String>());
-               }
-               
-               ArrayList<String> nGramHelper(List<String> str, 
ArrayList<String> nGrams) {
-                       if (str.size() == 0)
-                               return nGrams;
-                       
-                       for (int i = 0; i < str.size(); i++)
-                               nGrams.add(makeString(str.subList(0, i+1)));
-                       
-                       return nGramHelper(str.subList(1, str.size()), nGrams);
-               }
-               
-               String makeString(List<String> list) {
-                       StringBuffer sb = new StringBuffer();
-                       for (Iterator<String> it = list.iterator(); 
it.hasNext(); ) {
-                               sb.append(it.next());
-                               if (it.hasNext())
-                                       sb.append(" ");
-                       }
-                       return sb.toString();
-               }
-       }
-
-       
-       
-       
-       @Test
-       public void testBagFunctionWithFlattening() throws Exception{
-               PigServer pigServer = new PigServer(initString);
-               File queryLogFile = createFile(
-                                       new String[]{ 
-                                               "stanford\tdeer\tsighting",
-                                               "bush\tpresident",
-                                               "stanford\tbush",
-                                               "conference\tyahoo",
-                                               "world\tcup\tcricket",
-                                               "bush\twins",
-                                               "stanford\tpresident",
-                                       }
-                               );
-                               
-               File newsFile = createFile(
-                                       new String[]{
-                                               "deer seen at stanford",
-                                               "george bush visits stanford", 
-                                               "yahoo hosting a conference in 
the bay area", 
-                                               "who will win the world cup"
-                                       }
-                               );      
-               
-               Map<String, Integer> expectedResults = new HashMap<String, 
Integer>();
-               expectedResults.put("bush", 2);
-               expectedResults.put("stanford", 3);
-               expectedResults.put("world", 1);
-               expectedResults.put("conference", 1);
-               
-               pigServer.registerQuery("newsArticles = LOAD 'file:" + newsFile 
+ "' USING " + TextLoader.class.getName() + "();");
-           pigServer.registerQuery("queryLog = LOAD 'file:" + queryLogFile + 
"';");
-
-           pigServer.registerQuery("titleNGrams = FOREACH newsArticles 
GENERATE flatten(" + TitleNGrams.class.getName() + "(*));");
-           pigServer.registerQuery("cogrouped = COGROUP titleNGrams BY $0 
INNER, queryLog BY $0 INNER;");
-           pigServer.registerQuery("answer = FOREACH cogrouped GENERATE 
COUNT(queryLog),group;");
-               
-           Iterator<Tuple> iter = pigServer.openIterator("answer");
-           while(iter.hasNext()){
-               Tuple t = iter.next();
-               
assertEquals(expectedResults.get(t.getAtomField(1).strval()).doubleValue(),t.getAtomField(0).numval().doubleValue());
-           }
-       }
-       
-
-       
-       @Test
-       public void testSort() throws Exception{
-               testSortDistinct(false);
-       }
-       
-
-       @Test
-       public void testDistinct() throws Exception{
-               testSortDistinct(true);
-       }
-
-       private void testSortDistinct(boolean eliminateDuplicates) throws 
Exception{
-               int LOOP_SIZE = 1024*16;
-               File tmpFile = File.createTempFile("test", "txt");
-        PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
-        Random r = new Random();
-        for(int i = 0; i < LOOP_SIZE; i++) {
-            ps.println(r.nextInt(LOOP_SIZE/2) + "\t" + i);
-        }
-        ps.close(); 
-               
-               PigServer pig = new PigServer(initString);
-        String tmpOutputFile = FileLocalizer.getTemporaryPath(null, 
pig.getPigContext()).toString();
-               pig.registerQuery("A = LOAD 'file:" + tmpFile + "';");
-               if (eliminateDuplicates){
-                       pig.registerQuery("B = DISTINCT (FOREACH A GENERATE $0) 
PARALLEL 10;");
-               }else{
-                       pig.registerQuery("B = ORDER A BY $0 PARALLEL 10;");
-               }
-               pig.store("B", tmpOutputFile);
-               
-               pig.registerQuery("A = load '" + tmpOutputFile + "';");
-               Iterator<Tuple> iter = pig.openIterator("A");
-               int last = -1;
-               while (iter.hasNext()){
-                       Tuple t = iter.next();
-                       if (eliminateDuplicates){
-                               assertTrue(last < 
t.getAtomField(0).numval().intValue());
-                       }else{
-                               assertTrue(last <= 
t.getAtomField(0).numval().intValue());
-                               assertEquals(t.arity(), 2);
-                       }
-               }
-               
-       }
-       
-
-}
+package org.apache.pig.test;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.StringTokenizer;
+
+import org.junit.Test;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.PigServer;
+import org.apache.pig.builtin.BinStorage;
+import org.apache.pig.builtin.PigStorage;
+import org.apache.pig.builtin.TextLoader;
+import org.apache.pig.data.*;
+import org.apache.pig.impl.io.FileLocalizer;
+import org.apache.pig.impl.io.PigFile;
+
+import junit.framework.TestCase;
+
+public class TestEvalPipeline extends TestCase {
+    
+    String initString = "mapreduce";
+
+    TupleFactory mTf = TupleFactory.getInstance();
+    
+    
+    static public class MyBagFunction extends EvalFunc<DataBag>{
+        @Override
+        public DataBag exec(Tuple input) throws IOException {
+            TupleFactory tf = TupleFactory.getInstance();
+            DataBag output = BagFactory.getInstance().newDefaultBag();
+            output.add(tf.newTuple("a"));
+            output.add(tf.newTuple("a"));
+            output.add(tf.newTuple("a"));
+            return output;
+            
+        }
+    }
+    
+    
+    private File createFile(String[] data) throws Exception{
+        File f = File.createTempFile("tmp", "");
+        PrintWriter pw = new PrintWriter(f);
+        for (int i=0; i<data.length; i++){
+            pw.println(data[i]);
+        }
+        pw.close();
+        return f;
+    }
+    
+    @Test
+    public void testFunctionInsideFunction() throws Exception{
+        PigServer pigServer = new PigServer(initString);
+        
+        File f1 = createFile(new String[]{"a:1","b:1","a:1"});
+
+        pigServer.registerQuery("a = load 'file:" + f1 + "' using " + 
PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("b = foreach a generate '1'-'1'/'1';");
+        Iterator<Tuple> iter  = pigServer.openIterator("b");
+        
+        for (int i=0 ;i<3; i++){
+            assertEquals(DataType.toDouble(iter.next().get(0)), 0.0);
+        }
+        
+    }
+    
+    @Test
+    public void testJoin() throws Exception{
+        PigServer pigServer = new PigServer(initString);
+        
+        File f1 = createFile(new String[]{"a:1","b:1","a:1"});
+        File f2 = createFile(new String[]{"b","b","a"});
+        
+        pigServer.registerQuery("a = load 'file:" + f1 + "' using " + 
PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("b = load 'file:" + f2 + "';");
+        pigServer.registerQuery("c = cogroup a by $0, b by $0;");        
+        pigServer.registerQuery("d = foreach c generate 
flatten($1),flatten($2);");
+        
+        Iterator<Tuple> iter = pigServer.openIterator("d");
+        int count = 0;
+        while(iter.hasNext()){
+            Tuple t = iter.next();
+            assertTrue(t.get(0).toString().equals(t.get(2).toString()));
+            count++;
+        }
+        assertEquals(count, 4);
+    }
+    
+    @Test
+    public void testDriverMethod() throws Exception{
+        PigServer pigServer = new PigServer(initString);
+        File f = File.createTempFile("tmp", "");
+        PrintWriter pw = new PrintWriter(f);
+        pw.println("a");
+        pw.println("a");
+        pw.close();
+        pigServer.registerQuery("a = foreach (load 'file:" + f + "') generate 
'1', flatten(" + MyBagFunction.class.getName() + "(*));");
+        pigServer.registerQuery("b = foreach a generate $0, flatten($1);");
+        Iterator<Tuple> iter = pigServer.openIterator("a");
+        int count = 0;
+        while(iter.hasNext()){
+            Tuple t = iter.next();
+            assertTrue(t.get(0).toString().equals("1"));
+            assertTrue(t.get(1).toString().equals("a"));
+            count++;
+        }
+        assertEquals(count, 6);
+        f.delete();
+    }
+    
+    
+    @Test
+    public void testMapLookup() throws IOException{
+        PigServer pigServer = new PigServer(initString);
+        DataBag b = BagFactory.getInstance().newDefaultBag();
+        Map<Object, Object> colors = new HashMap<Object, Object>();
+        colors.put("apple","red");
+        colors.put("orange","orange");
+        
+        Map<Object, Object> weights = new HashMap<Object, Object>();
+        weights.put("apple","0.1");
+        weights.put("orange","0.3");
+        
+        Tuple t = mTf.newTuple();
+        t.append(colors);
+        t.append(weights);
+        b.add(t);
+        
+        String fileName = "file:"+File.createTempFile("tmp", "");
+        PigFile f = new PigFile(fileName);
+        f.store(b, new BinStorage(), pigServer.getPigContext());
+        
+        
+        pigServer.registerQuery("a = load '" + fileName + "' using 
BinStorage();");
+        pigServer.registerQuery("b = foreach a generate 
$0#'apple',flatten($1#'orange');");
+        Iterator<Tuple> iter = pigServer.openIterator("b");
+        t = iter.next();
+        assertEquals(t.get(0).toString(), "red");
+        assertEquals(DataType.toDouble(t.get(1)), 0.3);
+        assertFalse(iter.hasNext());
+    }
+    
+    
+    static public class TitleNGrams extends EvalFunc<DataBag> {
+        
+        @Override
+        public DataBag exec(Tuple input) throws IOException {    
+            DataBag output = BagFactory.getInstance().newDefaultBag();
+            String str = input.get(0).toString();
+            
+            String title = str;
+
+            if (title != null) {
+                List<String> nGrams = makeNGrams(title);
+                
+                for (Iterator<String> it = nGrams.iterator(); it.hasNext(); ) {
+                    Tuple t = TupleFactory.getInstance().newTuple(1);
+                    t.set(0, it.next());
+                    output.add(t);
+                }
+            }
+
+            return output;
+        }
+        
+        
+        List<String> makeNGrams(String str) {
+            List<String> tokens = new ArrayList<String>();
+            
+            StringTokenizer st = new StringTokenizer(str);
+            while (st.hasMoreTokens())
+                tokens.add(st.nextToken());
+            
+            return nGramHelper(tokens, new ArrayList<String>());
+        }
+        
+        ArrayList<String> nGramHelper(List<String> str, ArrayList<String> 
nGrams) {
+            if (str.size() == 0)
+                return nGrams;
+            
+            for (int i = 0; i < str.size(); i++)
+                nGrams.add(makeString(str.subList(0, i+1)));
+            
+            return nGramHelper(str.subList(1, str.size()), nGrams);
+        }
+        
+        String makeString(List<String> list) {
+            StringBuffer sb = new StringBuffer();
+            for (Iterator<String> it = list.iterator(); it.hasNext(); ) {
+                sb.append(it.next());
+                if (it.hasNext())
+                    sb.append(" ");
+            }
+            return sb.toString();
+        }
+    }
+
+    
+    
+    
+    @Test
+    public void testBagFunctionWithFlattening() throws Exception{
+        PigServer pigServer = new PigServer(initString);
+        File queryLogFile = createFile(
+                    new String[]{ 
+                        "stanford\tdeer\tsighting",
+                        "bush\tpresident",
+                        "stanford\tbush",
+                        "conference\tyahoo",
+                        "world\tcup\tcricket",
+                        "bush\twins",
+                        "stanford\tpresident",
+                    }
+                );
+                
+        File newsFile = createFile(
+                    new String[]{
+                        "deer seen at stanford",
+                        "george bush visits stanford", 
+                        "yahoo hosting a conference in the bay area", 
+                        "who will win the world cup"
+                    }
+                );    
+        
+        Map<String, Integer> expectedResults = new HashMap<String, Integer>();
+        expectedResults.put("bush", 2);
+        expectedResults.put("stanford", 3);
+        expectedResults.put("world", 1);
+        expectedResults.put("conference", 1);
+        
+        pigServer.registerQuery("newsArticles = LOAD 'file:" + newsFile + "' 
USING " + TextLoader.class.getName() + "();");
+        pigServer.registerQuery("queryLog = LOAD 'file:" + queryLogFile + 
"';");
+
+        pigServer.registerQuery("titleNGrams = FOREACH newsArticles GENERATE 
flatten(" + TitleNGrams.class.getName() + "(*));");
+        pigServer.registerQuery("cogrouped = COGROUP titleNGrams BY $0 INNER, 
queryLog BY $0 INNER;");
+        pigServer.registerQuery("answer = FOREACH cogrouped GENERATE 
COUNT(queryLog),group;");
+        
+        Iterator<Tuple> iter = pigServer.openIterator("answer");
+        while(iter.hasNext()){
+            Tuple t = iter.next();
+            
assertEquals(expectedResults.get(t.get(1).toString()).doubleValue(),(DataType.toDouble(t.get(0))).doubleValue());
+        }
+    }
+    
+
+    
+    @Test
+    public void testSort() throws Exception{
+        testSortDistinct(false);
+    }
+    
+
+    @Test
+    public void testDistinct() throws Exception{
+        testSortDistinct(true);
+    }
+
+    private void testSortDistinct(boolean eliminateDuplicates) throws 
Exception{
+        int LOOP_SIZE = 1024*16;
+        File tmpFile = File.createTempFile("test", "txt");
+        PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
+        Random r = new Random();
+        for(int i = 0; i < LOOP_SIZE; i++) {
+            ps.println(r.nextInt(LOOP_SIZE/2) + "\t" + i);
+        }
+        ps.close(); 
+        
+        PigServer pig = new PigServer(initString);
+        String tmpOutputFile = FileLocalizer.getTemporaryPath(null, 
pig.getPigContext()).toString();
+        pig.registerQuery("A = LOAD 'file:" + tmpFile + "';");
+        if (eliminateDuplicates){
+            pig.registerQuery("B = DISTINCT (FOREACH A GENERATE $0) PARALLEL 
10;");
+        }else{
+            pig.registerQuery("B = ORDER A BY $0 PARALLEL 10;");
+        }
+        pig.store("B", tmpOutputFile);
+        
+        pig.registerQuery("A = load '" + tmpOutputFile + "';");
+        Iterator<Tuple> iter = pig.openIterator("A");
+        int last = -1;
+        while (iter.hasNext()){
+            Tuple t = iter.next();
+            if (eliminateDuplicates){
+System.out.println("HERE " + DataType.findType(t.get(0)));
+                assertTrue(last < Integer.valueOf(t.get(0).toString()));
+            }else{
+                assertTrue(last <= DataType.toDouble(t.get(0)));
+                assertEquals(t.size(), 2);
+            }
+        }
+        
+    }
+    
+
+}

Modified: 
incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpNumeric.java
URL: 
http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpNumeric.java?rev=630357&r1=630356&r2=630357&view=diff
==============================================================================
--- 
incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpNumeric.java 
(original)
+++ 
incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpNumeric.java 
Fri Feb 22 15:44:37 2008
@@ -59,12 +59,12 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            Double first = t.getAtomField(0).numval();
-            Double second = t.getAtomField(1).numval();
+            Double first = Double.valueOf(t.get(0).toString());
+            Double second = Double.valueOf(t.get(1).toString());
             assertTrue(first.equals(second));
         
-            String sfirst = t.getAtomField(0).strval();
-            String ssecond = t.getAtomField(1).strval();
+            String sfirst = t.get(0).toString();
+            String ssecond = t.get(1).toString();
             assertFalse(sfirst.equals(ssecond));
         }
     }
@@ -91,8 +91,8 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            Double first = t.getAtomField(0).numval();
-            Double second = t.getAtomField(1).numval();
+            Double first = Double.valueOf(t.get(0).toString());
+            Double second = Double.valueOf(t.get(1).toString());
             assertFalse(first.equals(second));
         }
     }
@@ -120,8 +120,8 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            Double first = t.getAtomField(0).numval();
-            Double second = t.getAtomField(1).numval();
+            Double first = Double.valueOf(t.get(0).toString());
+            Double second = Double.valueOf(t.get(1).toString());
             assertTrue(first.compareTo(second) > 0);
         }
     }
@@ -145,14 +145,14 @@
         int count =0;
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            Double first = t.getAtomField(0).numval();
+            Double first = Double.valueOf(t.get(0).toString());
             if (first == 1)
                count++;
             else
                assertTrue(first == 0);
             
         }
-        assertTrue(count == 10);
+        assertEquals("expected count of 10", 10, count);
     }
     
     
@@ -175,12 +175,12 @@
         int count =0;
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            Double first = t.getAtomField(0).numval();
+            Double first = Double.valueOf(t.get(0).toString());
             count+=first;
                assertTrue(first == 1 || first == 2 || first == 0);
             
         }
-        assertTrue(count == 15);
+        assertEquals("expected count of 15", 15, count);
     }
     
     
@@ -208,8 +208,8 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            Double first = t.getAtomField(0).numval();
-            Double second = t.getAtomField(1).numval();
+            Double first = Double.valueOf(t.get(0).toString());
+            Double second = Double.valueOf(t.get(1).toString());
             assertTrue(first.compareTo(second) < 0);
         }
        
@@ -241,8 +241,8 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            Double first = t.getAtomField(0).numval();
-            Double second = t.getAtomField(1).numval();
+            Double first = Double.valueOf(t.get(0).toString());
+            Double second = Double.valueOf(t.get(1).toString());
             assertTrue(first.compareTo(second) >= 0);
         }
     }
@@ -272,8 +272,8 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            Double first = t.getAtomField(0).numval();
-            Double second = t.getAtomField(1).numval();
+            Double first = Double.valueOf(t.get(0).toString());
+            Double second = Double.valueOf(t.get(1).toString());
             assertTrue(first.compareTo(second) <= 0);
         }
     }

Modified: 
incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpString.java
URL: 
http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpString.java?rev=630357&r1=630356&r2=630357&view=diff
==============================================================================
--- 
incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpString.java 
(original)
+++ 
incubator/pig/branches/types/test/org/apache/pig/test/TestFilterOpString.java 
Fri Feb 22 15:44:37 2008
@@ -59,8 +59,8 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            String first = t.getAtomField(0).strval();
-            String second = t.getAtomField(1).strval();
+            String first = t.get(0).toString();
+            String second = t.get(1).toString();
             assertTrue(first.equals(second));
         }
     }
@@ -88,8 +88,8 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            String first = t.getAtomField(0).strval();
-            String second = t.getAtomField(1).strval();
+            String first = t.get(0).toString();
+            String second = t.get(1).toString();
             assertFalse(first.equals(second));
         }
     }
@@ -117,8 +117,8 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            String first = t.getAtomField(0).strval();
-            String second = t.getAtomField(1).strval();
+            String first = t.get(0).toString();
+            String second = t.get(1).toString();
             assertTrue(first.compareTo(second) > 0);
         }
     }
@@ -151,8 +151,8 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            String first = t.getAtomField(0).strval();
-            String second = t.getAtomField(1).strval();
+            String first = t.get(0).toString();
+            String second = t.get(1).toString();
             assertTrue(first.compareTo(second) >= 0);
         }
     }
@@ -181,8 +181,8 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            String first = t.getAtomField(0).strval();
-            String second = t.getAtomField(1).strval();
+            String first = t.get(0).toString();
+            String second = t.get(1).toString();
             assertTrue(first.compareTo(second) < 0);
         }
     }
@@ -213,8 +213,8 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            String first = t.getAtomField(0).strval();
-            String second = t.getAtomField(1).strval();
+            String first = t.get(0).toString();
+            String second = t.get(1).toString();
             assertTrue(first.compareTo(second) <= 0);
         }
     }

Modified: 
incubator/pig/branches/types/test/org/apache/pig/test/TestInfixArithmetic.java
URL: 
http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestInfixArithmetic.java?rev=630357&r1=630356&r2=630357&view=diff
==============================================================================
--- 
incubator/pig/branches/types/test/org/apache/pig/test/TestInfixArithmetic.java 
(original)
+++ 
incubator/pig/branches/types/test/org/apache/pig/test/TestInfixArithmetic.java 
Fri Feb 22 15:44:37 2008
@@ -26,6 +26,7 @@
 
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.PigStorage;
+import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 
 import junit.framework.TestCase;
@@ -51,8 +52,8 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            Double first = t.getAtomField(0).numval();
-            Double second = t.getAtomField(1).numval();
+            Double first = DataType.toDouble(t.get(0));
+            Double second = DataType.toDouble(t.get(1));
             assertTrue(second.equals(first + first));
         }
     }
@@ -73,7 +74,7 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            Double second = t.getAtomField(1).numval();
+            Double second = DataType.toDouble(t.get(1));
             assertTrue(second.equals(0.0));
         }
     }
@@ -94,8 +95,8 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            Double first = t.getAtomField(0).numval();
-            Double second = t.getAtomField(1).numval();
+            Double first = DataType.toDouble(t.get(0));
+            Double second = DataType.toDouble(t.get(1));
             assertTrue(second.equals(first * first));
         }
     }
@@ -116,7 +117,7 @@
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
-            Double second = t.getAtomField(1).numval();
+            Double second = DataType.toDouble(t.get(1));
             assertTrue(second.equals(1.0));
         }
     }

Modified: 
incubator/pig/branches/types/test/org/apache/pig/test/TestLargeFile.java
URL: 
http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestLargeFile.java?rev=630357&r1=630356&r2=630357&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestLargeFile.java 
(original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestLargeFile.java 
Fri Feb 22 15:44:37 2008
@@ -20,6 +20,7 @@
 
 import org.apache.pig.PigServer;
 
+import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 
 import org.apache.pig.impl.io.FileLocalizer;
@@ -112,8 +113,8 @@
                
                while(B.hasNext()) {
                        Tuple temp = B.next();
-                       int index = temp.getAtomField(0).numval().intValue();
-                       int value = temp.getAtomField(1).numval().intValue();
+                       int index = DataType.toInteger(temp.get(0));
+                       int value = DataType.toInteger(temp.get(1));
                        System.out.println("COUNT [" + index + "] = " + 
COUNT[index] + " B[" + index + "] = " + value);
                        
                        assertEquals(COUNT[index].intValue(), value);
@@ -132,12 +133,12 @@
                Iterator <Tuple> B = pig.openIterator("B");
                
                if(B.hasNext()) {
-                       N = B.next().getAtomField(0).numval().intValue();
+                       N = DataType.toInteger(B.next().get(0));
                }
                
                while(B.hasNext()) {
                        int flag = 0;
-                       Nplus1 = B.next().getAtomField(0).numval().intValue();
+                       Nplus1 = DataType.toInteger(B.next().get(0));
                        if(Nplus1 >= N) {
                                flag = 1;
                        }
@@ -171,7 +172,7 @@
         }
                
                while(B.hasNext()) {
-                       int temp = B.next().getAtomField(0).numval().intValue();
+                       int temp = DataType.toInteger(B.next().get(0));
                        COUNT_Test[temp] ++;
                }
                

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=630357&r1=630356&r2=630357&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
 Fri Feb 22 15:44:37 2008
@@ -30,6 +30,7 @@
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.PigServer.ExecType;
 import org.apache.pig.impl.builtin.ShellBagEvalFunc;
@@ -186,8 +187,9 @@
 
     static public class TestApplyFunc extends org.apache.pig.EvalFunc<Tuple> {
         @Override
-        public void exec(Tuple input, Tuple output) throws IOException {
-            output.appendTuple(input);
+        public Tuple exec(Tuple input) throws IOException {
+            Tuple output = TupleFactory.getInstance().newTuple(input.getAll());
+            return output;
         }
     }
 

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=630357&r1=630356&r2=630357&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 
Fri Feb 22 15:44:37 2008
@@ -38,8 +38,11 @@
 import org.apache.pig.StoreFunc;
 import org.apache.pig.PigServer.ExecType;
 import org.apache.pig.builtin.COUNT;
+import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.DataBag;
+import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.io.BufferedPositionedInputStream;
 import org.apache.pig.impl.PigContext;
@@ -64,7 +67,7 @@
         Iterator it = pig.openIterator("asdf_id");
         tmpFile.delete();
         Tuple t = (Tuple)it.next();
-        Double count = t.getAtomField(0).numval();
+        Double count = DataType.toDouble(t.get(0));
         assertEquals(count, (double)LOOP_COUNT);
     }
     
@@ -75,22 +78,26 @@
                this.field0 = field0;
        }
         @Override
-               public void exec(Tuple input, DataBag output) throws 
IOException {
-            Iterator<Tuple> it = (input.getBagField(0)).iterator();
+               public DataBag exec(Tuple input) throws IOException {
+            DataBag output = BagFactory.getInstance().newDefaultBag();
+            Iterator<Tuple> it = (DataType.toBag(input.get(0))).iterator();
             while(it.hasNext()) {
                 Tuple t = it.next();
-                Tuple newT = new Tuple(2);
-                newT.setField(0, field0);
-                newT.setField(1, t.getField(0).toString());
+                Tuple newT = TupleFactory.getInstance().newTuple(2);
+                newT.set(0, field0);
+                newT.set(1, t.get(0).toString());
                 output.add(newT);
             }
-          
+
+            return output;
         }
     }
     static public class MyGroup extends EvalFunc<Tuple> {
         @Override
-        public void exec(Tuple input, Tuple output) throws IOException{
-            output.appendField(new DataAtom("g"));
+        public Tuple exec(Tuple input) throws IOException{
+            Tuple output = TupleFactory.getInstance().newTuple(1);
+            output.set(0, new String("g"));
+            return output;
         }
     }
     static public class MyStorage implements LoadFunc, StoreFunc {
@@ -100,7 +107,7 @@
         }
                public Tuple getNext() throws IOException {
             if (count < COUNT) {
-                Tuple t = new Tuple(Integer.toString(count++));
+                Tuple t = 
TupleFactory.getInstance().newTuple(Integer.toString(count++));
                 return t;
             }
             return null;
@@ -161,8 +168,8 @@
         int count = 0;
         while(it.hasNext()) {
             t = (Tuple) it.next();
-            assertEquals(t.getField(0).toString(), "Got");
-            Integer.parseInt(t.getField(1).toString());
+            assertEquals(t.get(0).toString(), "Got");
+            Integer.parseInt(t.get(1).toString());
             count++;
         }
         assertEquals(count, MyStorage.COUNT);
@@ -187,8 +194,8 @@
         int count = 0;
         while(it.hasNext()) {
             t = (Tuple) it.next();
-            assertEquals("foo", t.getField(0).toString());
-            Integer.parseInt(t.getField(1).toString());
+            assertEquals("foo", t.get(0).toString());
+            Integer.parseInt(t.get(1).toString());
             count++;
         }
         assertEquals(count, MyStorage.COUNT);
@@ -219,7 +226,7 @@
                pig.deleteFile("/tmp/test_createNewRelation");
         System.out.println("testing create new relation");
         pig.newRelation("new_rel");
-        pig.insertTuple("new_rel", new Tuple("hello"));
+        pig.insertTuple("new_rel", 
TupleFactory.getInstance().newTuple("hello"));
         pig.store("new_rel", "/tmp/test_createNewRelation");
         assertTrue(pig.existsFile("/tmp/test_createNewRelation"));
     }

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestOrderBy.java
URL: 
http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestOrderBy.java?rev=630357&r1=630356&r2=630357&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestOrderBy.java 
(original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestOrderBy.java Fri 
Feb 22 15:44:37 2008
@@ -28,6 +28,7 @@
 import org.junit.Test;
 
 import org.apache.pig.PigServer;
+import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 
 public class TestOrderBy extends TestCase {
@@ -66,7 +67,7 @@
         int col = (descending ? 1 : 0);
         for(int i = 0; i < DATALEN; i++) {
             Tuple t = (Tuple)it.next();
-            int value = t.getAtomField(1).numval().intValue();
+            int value = DataType.toInteger(t.get(1));
 //            System.out.println("" + i + "," + DATA[0][i] + "," + DATA[1][i] 
+ "," + value);
             assertEquals(Integer.parseInt(DATA[col][i]), value);
         }

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestPi.java
URL: 
http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestPi.java?rev=630357&r1=630356&r2=630357&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestPi.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestPi.java Fri Feb 
22 15:44:37 2008
@@ -20,6 +20,7 @@
 
 import org.apache.pig.PigServer;
 
+import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 
 import org.apache.pig.impl.io.FileLocalizer;
@@ -126,8 +127,8 @@
                Iterator <Tuple> InCircle = pig.openIterator("K");
 
                
-               int totalPoints = 
Total.next().getAtomField(0).numval().intValue();
-               int inCirclePoints = 
InCircle.next().getAtomField(0).numval().intValue();
+               int totalPoints = DataType.toInteger(Total.next().get(0));
+               int inCirclePoints = DataType.toInteger(InCircle.next().get(0));
 
                System.out.println("Value of PI = " + 4 * (double)inCircle / 
(double)total);
                System.out.println("Value of PI (From Test data) = " + 4 * 
(double)inCirclePoints / (double)totalPoints);
@@ -137,7 +138,7 @@
                
                while(lengthTest.hasNext()) {
                        Tuple temp = lengthTest.next();
-                       totalLengthTest += 
temp.getAtomField(0).toString().length();
+                       totalLengthTest += temp.get(0).toString().length();
                }
                
                assertEquals(totalPoints, total);

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestPigFile.java
URL: 
http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestPigFile.java?rev=630357&r1=630356&r2=630357&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestPigFile.java 
(original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestPigFile.java Fri 
Feb 22 15:44:37 2008
@@ -22,6 +22,7 @@
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.Iterator;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
 
@@ -51,16 +52,17 @@
         System.out.println("Generating PigFile test data...");
 
         Random rand = new Random();
+        byte[] r = new byte[10];
 
-        Tuple t = new Tuple(10);
+        Tuple t = TupleFactory.getInstance().newTuple(10);
         for (int i = 0, j = 0; i < 10000; i++, j++) {
-            int r = rand.nextInt();
+            rand.nextBytes(r);
             if (j == 10) {
                 bag.add(t);
-                t = new Tuple(10);
+                t = TupleFactory.getInstance().newTuple(10);
                 j = 0;
             }
-            t.setField(j, r);
+            t.set(j, new DataByteArray(r));
 
         }
         System.out.println("Done.");
@@ -112,17 +114,32 @@
        
        throw new RuntimeException("Shouldn't reach here.");
     }
+
+    private char[] letters = {'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j',
+        'k', 'l', 'm', 'n', 'o', 'p', 'q', 'r', 's', 't', 'u', 'v', 'w', 
+        'x', 'y', 'z'};
     
-    private Integer getRandomDataAtom(){
-       return new Integer(rand.nextInt());
+    private DataByteArray getRandomDataAtom(){
+        /*
+        byte[] bytes = new byte[10];
+       rand.nextBytes(bytes);
+       //return new DataByteArray(bytes);
+       return new DataByteArray("Abc");
+        */
+        String s = new String();
+        for (int i = 0; i < 10; i++) {
+            s += letters[rand.nextInt(26)];
+        }
+        return new DataByteArray(s);
+
     }
     
     private Tuple getRandomTuple(int nestingLevel) throws IOException{
        
        int cardinality = rand.nextInt(2)+1;
-       Tuple t = new Tuple(cardinality);
+       Tuple t = TupleFactory.getInstance().newTuple(cardinality);
        for (int i=0; i<cardinality; i++)
-               t.setField(i, getRandomDatum(nestingLevel+1));
+               t.set(i, getRandomDatum(nestingLevel+1));
        return t;
     }
     
@@ -139,9 +156,9 @@
     
     private Map<Object, Object> getRandomMap(int nestingLevel) throws 
IOException{
        int cardinality = rand.nextInt(2)+1;
-       Map<Object, Object> m = new Map<Object, Object>();
+       Map<Object, Object> m = new HashMap<Object, Object>();
        for (int i=0; i<cardinality; i++){
-               
m.put(getRandomDataAtom().strval(),getRandomDatum(nestingLevel+1));
+               
m.put(getRandomDataAtom().toString(),getRandomDatum(nestingLevel+1));
        }
        return m;
     }

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestStore.java
URL: 
http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestStore.java?rev=630357&r1=630356&r2=630357&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestStore.java 
(original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestStore.java Fri 
Feb 22 15:44:37 2008
@@ -15,100 +15,101 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.pig.test;
-
-import java.io.File;
-import java.io.PrintWriter;
-import java.util.Iterator;
-
-import org.apache.pig.PigServer;
-import org.apache.pig.data.Tuple;
-import org.apache.pig.impl.io.FileLocalizer;
-
-import junit.framework.TestCase;
-
-public class TestStore extends TestCase {
-
-       private String initString = "mapreduce";
-       private int LOOP_COUNT = 1024;
-       
-       String fileName;
-       String tmpFile1, tmpFile2;
-       PigServer pig;
-       
-       public void testSingleStore() throws Exception{
-               pig.registerQuery("A = load " + fileName + ";");
-               
-               pig.store("A", tmpFile1);
-               
-               pig.registerQuery("B = load " + tmpFile1 + ";");
-               Iterator<Tuple> iter  = pig.openIterator("B");
-               
-               int i =0;
-               while (iter.hasNext()){
-                       Tuple t = iter.next();
-                       assertEquals(t.getAtomField(0).numval().intValue(),i);
-                       assertEquals(t.getAtomField(1).numval().intValue(),i);
-                       i++;
-               }
-       }
-       
-       public void testMultipleStore() throws Exception{
-               pig.registerQuery("A = load " + fileName + ";");
-               
-               pig.store("A", tmpFile1);
-               
-               pig.registerQuery("B = foreach (group A by $0) generate $0, 
SUM($1);");
-               pig.store("B", tmpFile2);
-               pig.registerQuery("C = load " + tmpFile2 + ";");
-               Iterator<Tuple> iter  = pig.openIterator("C");
-               
-               int i =0;
-               while (iter.hasNext()){
-                       Tuple t = iter.next();
-                       i++;
-                       
-               }
-               
-               assertEquals(LOOP_COUNT, i);
-               
-       }
-       
-       public void testStoreWithMultipleMRJobs() throws Exception{
-               pig.registerQuery("A = load " + fileName + ";");                
-               pig.registerQuery("B = foreach (group A by $0) generate $0, 
SUM($1);");
-               pig.registerQuery("C = foreach (group B by $0) generate $0, 
SUM($1);");
-               pig.registerQuery("D = foreach (group C by $0) generate $0, 
SUM($1);");
-
-               pig.store("D", tmpFile2);
-               pig.registerQuery("E = load " + tmpFile2 + ";");
-               Iterator<Tuple> iter  = pig.openIterator("E");
-               
-               int i =0;
-               while (iter.hasNext()){
-                       Tuple t = iter.next();
-                       i++;
-               }
-               
-               assertEquals(LOOP_COUNT, i);
-               
-       }
-
-       
-       @Override
-       protected void setUp() throws Exception {
-               super.setUp();
-               File f = File.createTempFile("tmp", "");
-               PrintWriter pw = new PrintWriter(f);
-               for (int i=0;i<LOOP_COUNT; i++){
-                       pw.println(i + "\t" + i);
-               }
-               pw.close();
-               pig = new PigServer(initString);
-               fileName = "'" + FileLocalizer.hadoopify(f.toString(), 
pig.getPigContext()) + "'";
-               tmpFile1 = "'" + FileLocalizer.getTemporaryPath(null, 
pig.getPigContext()).toString() + "'";
-               tmpFile2 = "'" + FileLocalizer.getTemporaryPath(null, 
pig.getPigContext()).toString() + "'";
-               f.delete();
-       }
-       
-}
+package org.apache.pig.test;
+
+import java.io.File;
+import java.io.PrintWriter;
+import java.util.Iterator;
+
+import org.apache.pig.PigServer;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.io.FileLocalizer;
+
+import junit.framework.TestCase;
+
+public class TestStore extends TestCase {
+
+    private String initString = "mapreduce";
+    private int LOOP_COUNT = 1024;
+    
+    String fileName;
+    String tmpFile1, tmpFile2;
+    PigServer pig;
+    
+    public void testSingleStore() throws Exception{
+        pig.registerQuery("A = load " + fileName + ";");
+        
+        pig.store("A", tmpFile1);
+        
+        pig.registerQuery("B = load " + tmpFile1 + ";");
+        Iterator<Tuple> iter  = pig.openIterator("B");
+        
+        int i =0;
+        while (iter.hasNext()){
+            Tuple t = iter.next();
+            assertEquals(DataType.toInteger(t.get(0)).intValue(),i);
+            assertEquals(DataType.toInteger(t.get(1)).intValue(),i);
+            i++;
+        }
+    }
+    
+    public void testMultipleStore() throws Exception{
+        pig.registerQuery("A = load " + fileName + ";");
+        
+        pig.store("A", tmpFile1);
+        
+        pig.registerQuery("B = foreach (group A by $0) generate $0, SUM($1);");
+        pig.store("B", tmpFile2);
+        pig.registerQuery("C = load " + tmpFile2 + ";");
+        Iterator<Tuple> iter  = pig.openIterator("C");
+        
+        int i =0;
+        while (iter.hasNext()){
+            Tuple t = iter.next();
+            i++;
+            
+        }
+        
+        assertEquals(LOOP_COUNT, i);
+        
+    }
+    
+    public void testStoreWithMultipleMRJobs() throws Exception{
+        pig.registerQuery("A = load " + fileName + ";");        
+        pig.registerQuery("B = foreach (group A by $0) generate $0, SUM($1);");
+        pig.registerQuery("C = foreach (group B by $0) generate $0, SUM($1);");
+        pig.registerQuery("D = foreach (group C by $0) generate $0, SUM($1);");
+
+        pig.store("D", tmpFile2);
+        pig.registerQuery("E = load " + tmpFile2 + ";");
+        Iterator<Tuple> iter  = pig.openIterator("E");
+        
+        int i =0;
+        while (iter.hasNext()){
+            Tuple t = iter.next();
+            i++;
+        }
+        
+        assertEquals(LOOP_COUNT, i);
+        
+    }
+
+    
+    @Override
+    protected void setUp() throws Exception {
+        super.setUp();
+        File f = File.createTempFile("tmp", "");
+        PrintWriter pw = new PrintWriter(f);
+        for (int i=0;i<LOOP_COUNT; i++){
+            pw.println(i + "\t" + i);
+        }
+        pw.close();
+        pig = new PigServer(initString);
+        fileName = "'" + FileLocalizer.hadoopify(f.toString(), 
pig.getPigContext()) + "'";
+        tmpFile1 = "'" + FileLocalizer.getTemporaryPath(null, 
pig.getPigContext()).toString() + "'";
+        tmpFile2 = "'" + FileLocalizer.getTemporaryPath(null, 
pig.getPigContext()).toString() + "'";
+        f.delete();
+    }
+    
+}

Modified: incubator/pig/branches/types/test/org/apache/pig/test/Util.java
URL: 
http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/Util.java?rev=630357&r1=630356&r2=630357&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/Util.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/Util.java Fri Feb 22 
15:44:37 2008
@@ -26,14 +26,21 @@
     // =================
     static public Tuple loadFlatTuple(Tuple t, int[] input) throws IOException 
{
         for (int i = 0; i < input.length; i++) {
-            t.setField(i, input[i]);
+            t.set(i, new Integer(input[i]));
         }
         return t;
     }
 
     static public Tuple loadTuple(Tuple t, String[] input) throws IOException {
         for (int i = 0; i < input.length; i++) {
-            t.setField(i, input[i]);
+            t.set(i, input[i]);
+        }
+        return t;
+    }
+
+    static public Tuple loadTuple(Tuple t, DataByteArray[] input) throws 
IOException {
+        for (int i = 0; i < input.length; i++) {
+            t.set(i, input[i]);
         }
         return t;
     }
@@ -41,20 +48,32 @@
     static public Tuple loadNestTuple(Tuple t, int[] input) throws IOException 
{
         DataBag bag = BagFactory.getInstance().newDefaultBag();
         for(int i = 0; i < input.length; i++) {
-            Tuple f = new Tuple(1);
-            f.setField(0, input[i]);
+            Tuple f = TupleFactory.getInstance().newTuple(1);
+            f.set(0, input[i]);
+            bag.add(f);
+        }
+        t.set(0, bag);
+        return t;
+    }
+
+    static public Tuple loadNestTuple(Tuple t, long[] input) throws 
IOException {
+        DataBag bag = BagFactory.getInstance().newDefaultBag();
+        for(int i = 0; i < input.length; i++) {
+            Tuple f = TupleFactory.getInstance().newTuple(1);
+            f.set(0, new Long(input[i]));
             bag.add(f);
         }
-        t.setField(0, bag);
+        t.set(0, bag);
         return t;
     }
 
+
     static public Tuple loadNestTuple(Tuple t, int[][] input) throws 
IOException {
         for (int i = 0; i < input.length; i++) {
             DataBag bag = BagFactory.getInstance().newDefaultBag();
-            Tuple f = loadFlatTuple(new Tuple(input[i].length), input[i]);
+            Tuple f = 
loadFlatTuple(TupleFactory.getInstance().newTuple(input[i].length), input[i]);
             bag.add(f);
-            t.setField(i, bag);
+            t.set(i, bag);
         }
         return t;
     }
@@ -62,10 +81,11 @@
     static public Tuple loadTuple(Tuple t, String[][] input) throws 
IOException {
         for (int i = 0; i < input.length; i++) {
             DataBag bag = BagFactory.getInstance().newDefaultBag();
-            Tuple f = loadTuple(new Tuple(input[i].length), input[i]);
+            Tuple f = 
loadTuple(TupleFactory.getInstance().newTuple(input[i].length), input[i]);
             bag.add(f);
-            t.setField(i, bag);
+            t.set(i, bag);
         }
         return t;
     }
+
 }


Reply via email to