You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ol...@apache.org on 2010/08/31 02:54:12 UTC

svn commit: r991051 - in /hadoop/pig/trunk: ./ src/org/apache/pig/builtin/ test/ test/org/apache/pig/test/

Author: olga
Date: Tue Aug 31 00:54:11 2010
New Revision: 991051

URL: http://svn.apache.org/viewvc?rev=991051&view=rev
Log:
PIG-1563: some of string functions don't work on bytearrays (olgan)

Added:
    hadoop/pig/trunk/src/org/apache/pig/builtin/STRSPLIT.java
    hadoop/pig/trunk/test/org/apache/pig/test/TestStringUDFs.java
Removed:
    hadoop/pig/trunk/src/org/apache/pig/builtin/SPLIT.java
Modified:
    hadoop/pig/trunk/CHANGES.txt
    hadoop/pig/trunk/src/org/apache/pig/builtin/LAST_INDEX_OF.java
    hadoop/pig/trunk/src/org/apache/pig/builtin/REPLACE.java
    hadoop/pig/trunk/src/org/apache/pig/builtin/SUBSTRING.java
    hadoop/pig/trunk/src/org/apache/pig/builtin/TRIM.java
    hadoop/pig/trunk/test/commit-tests
    hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java
    hadoop/pig/trunk/test/org/apache/pig/test/Util.java

Modified: hadoop/pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=991051&r1=991050&r2=991051&view=diff
==============================================================================
--- hadoop/pig/trunk/CHANGES.txt (original)
+++ hadoop/pig/trunk/CHANGES.txt Tue Aug 31 00:54:11 2010
@@ -175,6 +175,8 @@ PIG-1309: Map-side Cogroup (ashutoshc)
 
 BUG FIXES
 
+PIG-1563: some of string functions don't work on bytearrays (olgan)
+
 PIG-1569: java properties not honored in case of properties such as
 stop.on.failure (rding)
 

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/LAST_INDEX_OF.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/LAST_INDEX_OF.java?rev=991051&r1=991050&r2=991051&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/LAST_INDEX_OF.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/LAST_INDEX_OF.java Tue Aug 31 00:54:11 2010
@@ -19,13 +19,17 @@
 package org.apache.pig.builtin;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.pig.FuncSpec;
 import org.apache.pig.EvalFunc;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.data.DataType;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 
 /**
  * string.INSTR implements eval function to search for the last occurrence of a string
@@ -67,4 +71,16 @@ public class LAST_INDEX_OF extends EvalF
         return new Schema(new Schema.FieldSchema(null, DataType.INTEGER));
     }
 
-}
\ No newline at end of file
+    /* (non-Javadoc)
+     * @see org.apache.pig.EvalFunc#getArgToFuncMapping()
+     */
+    @Override
+    public List<FuncSpec> getArgToFuncMapping() throws FrontendException {
+        List<FuncSpec> funcList = new ArrayList<FuncSpec>();
+        Schema s = new Schema();
+        s.add(new Schema.FieldSchema(null, DataType.CHARARRAY));
+        s.add(new Schema.FieldSchema(null, DataType.CHARARRAY));
+        funcList.add(new FuncSpec(this.getClass().getName(), s));
+        return funcList;
+    }
+}

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/REPLACE.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/REPLACE.java?rev=991051&r1=991050&r2=991051&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/REPLACE.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/REPLACE.java Tue Aug 31 00:54:11 2010
@@ -19,12 +19,15 @@
 package org.apache.pig.builtin;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.pig.EvalFunc;
+import org.apache.pig.FuncSpec;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.data.DataType;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
-
+import org.apache.pig.impl.logicalLayer.FrontendException;
 
 /**
  * REPLACE implements eval function to replace part of a string.
@@ -63,4 +66,17 @@ public class REPLACE extends EvalFunc<St
         return new Schema(new Schema.FieldSchema(null, DataType.CHARARRAY));
     }
 
-}
\ No newline at end of file
+    /* (non-Javadoc)
+     * @see org.apache.pig.EvalFunc#getArgToFuncMapping()
+     */
+    @Override
+    public List<FuncSpec> getArgToFuncMapping() throws FrontendException {
+        List<FuncSpec> funcList = new ArrayList<FuncSpec>();
+        Schema s = new Schema();
+        s.add(new Schema.FieldSchema(null, DataType.CHARARRAY));
+        s.add(new Schema.FieldSchema(null, DataType.CHARARRAY));
+        s.add(new Schema.FieldSchema(null, DataType.CHARARRAY));
+        funcList.add(new FuncSpec(this.getClass().getName(), s));
+        return funcList;
+    }
+}

Added: hadoop/pig/trunk/src/org/apache/pig/builtin/STRSPLIT.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/STRSPLIT.java?rev=991051&view=auto
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/STRSPLIT.java (added)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/STRSPLIT.java Tue Aug 31 00:54:11 2010
@@ -0,0 +1,71 @@
+/*
+ * 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.builtin;
+
+import java.io.IOException;
+
+import java.util.Arrays;
+import java.util.regex.PatternSyntaxException;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+
+/**
+ * Wrapper around Java's String.split<br>
+ * input tuple: first column is assumed to have a string to split;<br>
+ * the optional second column is assumed to have the delimiter or regex to split on;<br>
+ * if not provided, it's assumed to be '\s' (space)<br>
+ * the optional third column may provide a limit to the number of results.<br>
+ * If limit is not provided, 0 is assumed, as per Java's split().
+ */
+
+public class STRSPLIT extends EvalFunc<Tuple> {
+
+    private final static TupleFactory tupleFactory = TupleFactory.getInstance();
+
+    /**
+     * Wrapper around Java's String.split
+     * @param input tuple; first column is assumed to have a string to split;
+     * the optional second column is assumed to have the delimiter or regex to split on;<br>
+     * if not provided, it's assumed to be '\s' (space)
+     * the optional third column may provide a limit to the number of results.<br>
+     * If limit is not provided, 0 is assumed, as per Java's split().
+     * @exception java.io.IOException
+     */
+    public Tuple exec(Tuple input) throws IOException {
+        if (input == null || input.size() < 1)
+            return null;
+        try {
+            String source = (String) input.get(0);
+            String delim = (input.size() > 1 ) ? (String) input.get(1) : "\\s";
+            int length = (input.size() > 2) ? (Integer) input.get(2) : 0;
+            if (source == null || delim == null) {
+                return null;
+            }
+            String[] splits = source.split(delim, length); 
+            return tupleFactory.newTuple(Arrays.asList(splits));
+        } catch (ClassCastException e) {
+            log.warn("class cast exception at "+e.getStackTrace()[0]);
+        } catch (PatternSyntaxException e) {
+            log.warn(e.getMessage());
+        }
+        // this only happens if the try block did not complete normally
+        return null;
+    }
+}

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/SUBSTRING.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/SUBSTRING.java?rev=991051&r1=991050&r2=991051&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/SUBSTRING.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/SUBSTRING.java Tue Aug 31 00:54:11 2010
@@ -19,11 +19,15 @@
 package org.apache.pig.builtin;
 
 import java.io.IOException;
+import java.util.List;
+import java.util.ArrayList;
 
 import org.apache.pig.EvalFunc;
+import org.apache.pig.FuncSpec;
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 
 /**
  * SUBSTRING implements eval function to get a part of a string.
@@ -67,4 +71,17 @@ public class SUBSTRING extends EvalFunc<
         return new Schema(new Schema.FieldSchema(null, DataType.CHARARRAY));
     }
 
-}
\ No newline at end of file
+    /* (non-Javadoc)
+     * @see org.apache.pig.EvalFunc#getArgToFuncMapping()
+     */
+    @Override
+    public List<FuncSpec> getArgToFuncMapping() throws FrontendException {
+        List<FuncSpec> funcList = new ArrayList<FuncSpec>();
+        Schema s = new Schema();
+        s.add(new Schema.FieldSchema(null, DataType.CHARARRAY));
+        s.add(new Schema.FieldSchema(null, DataType.INTEGER));
+        s.add(new Schema.FieldSchema(null, DataType.INTEGER));
+        funcList.add(new FuncSpec(this.getClass().getName(), s));
+        return funcList;
+    }
+}

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/TRIM.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/TRIM.java?rev=991051&r1=991050&r2=991051&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/TRIM.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/TRIM.java Tue Aug 31 00:54:11 2010
@@ -18,12 +18,16 @@
 package org.apache.pig.builtin;
 
 import java.io.IOException;
+import java.util.List;
+import java.util.ArrayList;
 
 import org.apache.pig.EvalFunc;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.FuncSpec;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 
 /** 
  * Returns a string, with leading and trailing whitespace omitted. 
@@ -50,4 +54,16 @@ public class TRIM extends EvalFunc<Strin
     public Schema outputSchema(Schema input) {
         return new Schema(new Schema.FieldSchema(null, DataType.CHARARRAY));
     }
+
+    /* (non-Javadoc)
+     * @see org.apache.pig.EvalFunc#getArgToFuncMapping()
+     */
+    @Override
+    public List<FuncSpec> getArgToFuncMapping() throws FrontendException {
+        List<FuncSpec> funcList = new ArrayList<FuncSpec>();
+        funcList.add(new FuncSpec(this.getClass().getName(), new Schema(new Schema.FieldSchema(null, DataType.CHARARRAY))));
+
+        return funcList;
+    }
+
 }

Modified: hadoop/pig/trunk/test/commit-tests
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/commit-tests?rev=991051&r1=991050&r2=991051&view=diff
==============================================================================
--- hadoop/pig/trunk/test/commit-tests (original)
+++ hadoop/pig/trunk/test/commit-tests Tue Aug 31 00:54:11 2010
@@ -57,3 +57,4 @@
 **/TestTupleFormat.java
 **/TestTypeChecking.java
 **/TestTypeCheckingValidatorNoSchema.java
+**/TestStringUDFs.java

Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java?rev=991051&r1=991050&r2=991051&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java Tue Aug 31 00:54:11 2010
@@ -57,7 +57,7 @@ import org.apache.pig.builtin.REGEX_EXTR
 import org.apache.pig.builtin.REGEX_EXTRACT_ALL;
 import org.apache.pig.builtin.REPLACE;
 import org.apache.pig.builtin.SIZE;
-import org.apache.pig.builtin.SPLIT;
+import org.apache.pig.builtin.STRSPLIT;
 import org.apache.pig.builtin.SUBSTRING;
 import org.apache.pig.builtin.StringConcat;
 import org.apache.pig.builtin.StringSize;
@@ -1258,7 +1258,7 @@ public class TestBuiltin {
         output = strFunc.exec(input);
         assertTrue(output.equals(expected));
         
-        SPLIT splitter = new SPLIT();
+        STRSPLIT splitter = new STRSPLIT();
         Tuple test1 = TupleFactory.getInstance().newTuple(1);
         Tuple test2 = TupleFactory.getInstance().newTuple(2);
         Tuple test3 = TupleFactory.getInstance().newTuple(3);
@@ -1921,6 +1921,54 @@ public class TestBuiltin {
         Util.deleteFile(cluster, "testSFPig-input.txt");
         Util.deleteFile(cluster, "testSFPig-output.txt");
     }
+
+    /* This are e2e tests to make sure that function that maps
+     * arguments to class is properly setup. More comprehansive
+     * unit tests are done in TestStringUDFs
+     */
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testStringUDFs() throws Exception {
+        String inputStr = "amy smith ";
+        Util.createInputFile(cluster, "testStrUDFsIn.txt", new String[] {inputStr});
+
+        // test typed data
+        pigServer.registerQuery("A = load 'testStrUDFsIn.txt' as (name: chararray);");
+        pigServer.registerQuery("B = foreach A generate SUBSTRING(name, 0, 3), " +
+            "INDEXOF(name, 'a'), INDEXOF(name, 'a', 3), LAST_INDEX_OF(name, 'a'), REPLACE(name, 'a', 'b'), " +
+            "STRSPLIT(name), STRSPLIT(name, ' '), STRSPLIT(name, ' ', 0), TRIM(name);"); 
+
+        Iterator<Tuple> it = pigServer.openIterator("B"); 
+        assertTrue(it.hasNext());
+        Tuple t = it.next();
+        Tuple expected = Util.buildTuple("amy", "smith");
+        assertTrue(!it.hasNext()); 
+        assertEquals(9, t.size());
+        assertEquals("amy", t.get(0));
+        assertEquals(0, t.get(1));
+        assertEquals(-1, t.get(2));
+        assertEquals(0, t.get(3));
+        assertEquals("bmy smith ", t.get(4));
+        assertEquals(expected, t.get(5));
+        assertEquals(expected, t.get(6));
+        assertEquals(expected, t.get(7));
+        assertEquals("amy smith", t.get(8));
+    
+        // test untyped data
+        pigServer.registerQuery("A = load 'testStrUDFsIn.txt' as (name);");
+        pigServer.registerQuery("B = foreach A generate SUBSTRING(name, 0, 3), " +
+            "LAST_INDEX_OF(name, 'a'), REPLACE(name, 'a', 'b'), TRIM(name);"); 
+
+        it = pigServer.openIterator("B"); 
+        assertTrue(it.hasNext());
+        t = it.next();
+        assertTrue(!it.hasNext()); 
+        assertEquals(4, t.size());
+        assertEquals("amy", t.get(0));
+        assertEquals(0, t.get(1));
+        assertEquals("bmy smith ", t.get(2));
+        assertEquals("amy smith", t.get(3));
+    }
     
     @Test
     public void testTOKENIZE() throws Exception {

Added: hadoop/pig/trunk/test/org/apache/pig/test/TestStringUDFs.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestStringUDFs.java?rev=991051&view=auto
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestStringUDFs.java (added)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestStringUDFs.java Tue Aug 31 00:54:11 2010
@@ -0,0 +1,148 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.io.IOException;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.builtin.INDEXOF;
+import org.apache.pig.builtin.LAST_INDEX_OF;
+import org.apache.pig.builtin.REPLACE;
+import org.apache.pig.builtin.STRSPLIT;
+import org.apache.pig.builtin.SUBSTRING;
+import org.apache.pig.builtin.TRIM;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.junit.Test;
+
+public class TestStringUDFs {
+    private static final EvalFunc<String> stringSubstr_ = new SUBSTRING();
+    private static final EvalFunc<String> Substr_ = new SUBSTRING();
+
+    @Test
+    public void testStringSubstr() throws IOException {
+        Tuple testTuple = Util.buildTuple(null, 0, 2);
+        assertNull("null is null", stringSubstr_.exec(testTuple));
+
+        testTuple = Util.buildTuple("", 0, 2);
+        assertEquals("empty string", "", stringSubstr_.exec(testTuple));
+
+        testTuple = Util.buildTuple("abcde", 1, 3);
+        assertEquals("lowercase string", "bc", stringSubstr_.exec(testTuple));
+
+        testTuple = Util.buildTuple("abc", 0, 15);
+        assertEquals("uppercase string", "abc", stringSubstr_.exec(testTuple));
+    }
+
+    @Test
+    public void testIndexOf() throws IOException {
+        INDEXOF indexOf = new INDEXOF();
+        Tuple testTuple = Util.buildTuple("xyz", "");
+        assertEquals( ((Integer) "xyz".indexOf("")), indexOf.exec(testTuple));
+        
+        testTuple = Util.buildTuple(null, null);
+        assertNull(indexOf.exec(testTuple));
+        
+        testTuple = Util.buildTuple("xyz", "y");
+        assertEquals( ((Integer) "xyz".indexOf("y")), indexOf.exec(testTuple));
+        
+        testTuple = Util.buildTuple("xyz", "abc");
+        assertEquals( ((Integer) "xyz".indexOf("abc")), indexOf.exec(testTuple));
+    }
+    
+    @Test
+    public void testLastIndexOf() throws IOException {
+        LAST_INDEX_OF lastIndexOf = new LAST_INDEX_OF();
+        Tuple testTuple = Util.buildTuple("xyz", "");
+        assertEquals( ((Integer) "xyz".lastIndexOf("")), lastIndexOf.exec(testTuple));
+        
+        testTuple = Util.buildTuple(null, null);
+        assertNull(lastIndexOf.exec(testTuple));
+        
+        testTuple = Util.buildTuple("xyzyy", "y");
+        assertEquals( ((Integer) "xyzyy".lastIndexOf("y")), lastIndexOf.exec(testTuple));
+        
+        testTuple = Util.buildTuple("xyz", "abc");
+        assertEquals( ((Integer) "xyz".lastIndexOf("abc")), lastIndexOf.exec(testTuple));
+    }
+    
+    @Test
+    public void testReplace() throws IOException {
+        REPLACE replace = new REPLACE();
+        Tuple testTuple = Util.buildTuple("foobar", "z", "x");
+        assertEquals("foobar".replace("z", "x"), replace.exec(testTuple));
+        
+        testTuple = Util.buildTuple("foobar", "oo", "aa");
+        assertEquals("foobar".replace("oo", "aa"), replace.exec(testTuple));
+    }
+    
+    @Test
+    public void testTrim() throws IOException {
+        TRIM trim = new TRIM();
+        Tuple testTuple = Util.buildTuple("nospaces");
+        assertEquals("nospaces".trim(), trim.exec(testTuple));
+        
+        testTuple = Util.buildTuple("spaces    ");
+        assertEquals("spaces     ".trim(), trim.exec(testTuple));
+        
+        testTuple = TupleFactory.getInstance().newTuple();
+        assertNull(trim.exec(testTuple));
+    }
+    
+    @Test 
+    public void testSplit() throws IOException {
+        STRSPLIT splitter = new STRSPLIT();
+       // test no delims
+        Tuple testTuple = Util.buildTuple("foo", ":");
+        testTuple.set(0, "foo");
+        testTuple.set(1, ":");
+        Tuple splits = splitter.exec(testTuple);
+        assertEquals("no matches should return tuple with original string", 1, splits.size());
+        assertEquals("no matches should return tuple with original string", "foo", 
+                splits.get(0));
+        
+        // test default delimiter
+        testTuple = Util.buildTuple("f ooo bar");
+        splits = splitter.exec(testTuple);
+        assertEquals("split on default value ", 3, splits.size());
+        assertEquals("f", splits.get(0));
+        assertEquals("ooo", splits.get(1));
+        assertEquals("bar", splits.get(2));
+        
+        // test trimming of whitespace
+        testTuple = Util.buildTuple("foo bar  ");
+        splits = splitter.exec(testTuple);
+        assertEquals("whitespace trimmed if no length arg", 2, splits.size());
+        
+        // test forcing null matches with length param
+        testTuple = Util.buildTuple("foo bar   ", "\\s", 10);
+        splits = splitter.exec(testTuple);
+        assertEquals("length forces empty string matches on end", 5, splits.size());
+        
+        // test limiting results with limit
+        testTuple = Util.buildTuple("foo:bar:baz", ":", 2);
+        splits = splitter.exec(testTuple);
+        assertEquals(2, splits.size());
+        assertEquals("foo", splits.get(0));
+        assertEquals("bar:baz", splits.get(1));
+    }
+}

Modified: hadoop/pig/trunk/test/org/apache/pig/test/Util.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/Util.java?rev=991051&r1=991050&r2=991051&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/Util.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/Util.java Tue Aug 31 00:54:11 2010
@@ -76,6 +76,9 @@ import org.apache.pig.newplan.logical.op
 import org.apache.pig.tools.grunt.Grunt;
 import org.apache.pig.tools.grunt.GruntParser;
 
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+
 public class Util {
     private static BagFactory mBagFactory = BagFactory.getInstance();
     private static TupleFactory mTupleFactory = TupleFactory.getInstance();
@@ -143,8 +146,26 @@ public class Util {
             t.append(b[i]);
     }
     
-    
-    
+    static public Tuple buildTuple(Object... args) throws ExecException {
+        return TupleFactory.getInstance().newTupleNoCopy(Lists.newArrayList(args));
+    }
+
+    static public Tuple buildBinTuple(final Object... args) throws IOException {
+        return TupleFactory.getInstance().newTuple(Lists.transform(
+                Lists.newArrayList(args), new Function<Object, DataByteArray>() {
+                    public DataByteArray apply(Object o) {
+                        if (o == null) { 
+                            return null;
+                        }
+                        try {
+                            return new DataByteArray(DataType.toBytes(o));
+                        } catch (ExecException e) {
+                            return null;
+                        }
+                    }
+                }));
+    }
+
     static public <T>Tuple createTuple(T[] s)
     {
         Tuple t = mTupleFactory.newTuple();