You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by da...@apache.org on 2009/07/31 03:40:54 UTC

svn commit: r799485 - in /hadoop/pig/trunk/contrib: ./ piggybank/java/ piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/datetime/ piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/ piggybank/java/src/main/java/org/...

Author: daijy
Date: Fri Jul 31 01:40:53 2009
New Revision: 799485

URL: http://svn.apache.org/viewvc?rev=799485&view=rev
Log:
PIG-885: New UDFs for piggybank (Bin, Decode, LookupInFiles, RegexExtract, RegexMatch, HashFVN, DiffDate)

Added:
    hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/datetime/
    hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/datetime/DiffDate.java
    hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/
    hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/Bin.java
    hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/BinCond.java
    hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/Decode.java
    hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/HashFNV.java
    hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/LookupInFiles.java
    hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/RegexExtract.java
    hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/RegexMatch.java
    hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/datetime/
    hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/datetime/TestDiffDate.java
    hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/decode/
    hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/decode/TestDecode.java
    hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/
    hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestHashFNV.java
    hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestLookupInFiles.java
    hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestRegex.java
Modified:
    hadoop/pig/trunk/contrib/CHANGES.txt
    hadoop/pig/trunk/contrib/piggybank/java/build.xml

Modified: hadoop/pig/trunk/contrib/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/CHANGES.txt?rev=799485&r1=799484&r2=799485&view=diff
==============================================================================
--- hadoop/pig/trunk/contrib/CHANGES.txt (original)
+++ hadoop/pig/trunk/contrib/CHANGES.txt Fri Jul 31 01:40:53 2009
@@ -1,3 +1,4 @@
+PIG-885: New UDFs for piggybank (Bin, Decode, LookupInFiles, RegexExtract, RegexMatch, HashFVN, DiffDate) (daijy)
 PIG-868: added strin manipulation functions (bennies via olgan)
 PIG-273: addition of Top and SearchQuery UDFs (ankur via olgan)
 PIG-246: created UDF repository (olgan)

Modified: hadoop/pig/trunk/contrib/piggybank/java/build.xml
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/build.xml?rev=799485&r1=799484&r2=799485&view=diff
==============================================================================
--- hadoop/pig/trunk/contrib/piggybank/java/build.xml (original)
+++ hadoop/pig/trunk/contrib/piggybank/java/build.xml Fri Jul 31 01:40:53 2009
@@ -32,6 +32,7 @@
     <property name="build.docs" value="${build.dir}/docs" />
     <property name="build.javadoc" value="${build.docs}/api" />
     <property name="pigjar" value="../../../pig.jar" />
+    <property name="pigtest" value="../../../build/test/classes" />
     <property name="udfjar" value="piggybank.jar" />
     <property name="src.dir" value="src/main/java/org/apache/pig/piggybank" />
 
@@ -45,16 +46,19 @@
     <property name="test.timeout" value="900000" />
     <property name="test.junit.output.format" value="plain" />
     <property name="test.src.dir" value="src/test/java" />
+    <property name="junit.hadoop.conf" value="${user.home}/pigtest/conf/"/>
 
     <path id="pigudf.classpath">
         <pathelement location="${build.classes}"/>
         <pathelement location="${pigjar}"/>
+        <pathelement location="${pigtest}"/>
     </path>
 
     <path id="test.classpath">
         <pathelement location="${build.classes}"/>
         <pathelement location="${test.classes}"/>
         <pathelement location="${test.src.dir}"/>
+        <pathelement location="${junit.hadoop.conf}" />
         <path refid="pigudf.classpath"/>
     </path>
 
@@ -93,6 +97,7 @@
         <delete dir="${test.logs}"/>
         <mkdir dir="${test.logs}"/>
         <junit printsummary="yes" haltonfailure="no" fork="yes" maxmemory="256m" dir="${basedir}" timeout="${test.timeout}" errorProperty="tests.failed" failureProperty="tests.failed">        
+            <sysproperty key="hadoop.log.dir" value="${test.logs}"/>
             <classpath refid="test.classpath"/>
             <formatter type="${test.junit.output.format}" />
             <batchtest fork="yes" todir="${test.logs}" unless="testcase">

Added: hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/datetime/DiffDate.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/datetime/DiffDate.java?rev=799485&view=auto
==============================================================================
--- hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/datetime/DiffDate.java (added)
+++ hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/datetime/DiffDate.java Fri Jul 31 01:40:53 2009
@@ -0,0 +1,93 @@
+/*
+ * 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.piggybank.evaluation.datetime;
+
+import java.io.IOException;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.FuncSpec;
+import org.apache.pig.PigWarning;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.FrontendException;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+/**
+* <dl>
+* <dt><b>Syntax:</b></dt>
+* <dd><code>int DiffDate(String yyyymmdd1, String yyyymmdd2)</code>.</dd>
+* <dt><b>Input:</b></dt>
+* <dd><code>date string in "yyyymmdd" format</code>.</dd>
+* <dt><b>Output:</b></dt>
+* <dd><code>(date1-date2) in days, can be negative</code>.</dd>
+* </dl>
+*/
+
+public class DiffDate extends EvalFunc<Integer> {
+    static DateFormat df = new SimpleDateFormat("yyyyMMdd");
+    @Override
+    public Schema outputSchema(Schema input) {
+        try {
+            return new Schema(new Schema.FieldSchema(getSchemaName(this
+                    .getClass().getName().toLowerCase(), input),
+                    DataType.INTEGER));
+        } catch (Exception e) {
+            return null;
+        }
+    }
+    @Override
+    public Integer exec(Tuple input) throws IOException {
+        if (input.size()!=2) {
+            String msg = "DiffDate : Only 2 parameters are allowed.";
+            throw new IOException(msg);
+        }
+        String strDate1 = (String)input.get(0);
+        String strDate2 = (String)input.get(1);
+        
+        if (input.get(0)==null || input.get(1)==null)
+            return null;
+        
+        Date date1 = null;
+        Date date2 = null;
+        
+        try {
+            date1 = df.parse(strDate1);
+            date2 = df.parse(strDate2);
+        } catch (ParseException e) {
+            String msg = "DiffDate : Parameters have to be string in 'yyyymmdd' format.";
+            warn(msg, PigWarning.UDF_WARNING_1);
+            return null;
+        }
+        return (int)((date1.getTime() - date2.getTime())/(1000*60*60*24)); 
+    }
+    @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;
+    }
+}

Added: hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/Bin.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/Bin.java?rev=799485&view=auto
==============================================================================
--- hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/Bin.java (added)
+++ hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/Bin.java Fri Jul 31 01:40:53 2009
@@ -0,0 +1,93 @@
+/*
+ * 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.piggybank.evaluation.decode;
+
+import java.io.IOException;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.PigWarning;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+/**
+* <dl>
+* <dt><b>Syntax:</b></dt>
+* <dd><code>String Bin(arithmetic_expression, string1, ,..., stringN, sentinelN, default_string)</code>.</dd>
+* <dt><b>Logic:</b></dt>
+* <dd><code>if      (arithmetic_expression<=sentinel1) return string1; <br>
+* ...... <br>
+* else if (arithmetic_expression<=sentinelN) return stringN; <br>
+* else                                       return default_string; <br></code>
+* <br>
+* arithmetic_expression can only be numeric types.</dd>
+* </dl>
+*/
+
+public class Bin extends EvalFunc<String> {
+
+    int numParams = -1;
+    @Override
+    public Schema outputSchema(Schema input) {
+        try {
+            return new Schema(new Schema.FieldSchema(getSchemaName(this
+                    .getClass().getName().toLowerCase(), input),
+                    DataType.CHARARRAY));
+        } catch (Exception e) {
+            return null;
+        }
+    }
+
+    @Override
+    public String exec(Tuple tuple) throws IOException {
+        if (numParams == -1)  // not initialized
+        {
+            numParams = tuple.size();
+            if (numParams <= 2) {
+                String msg = "Bin : An expression & atleast a default string are required.";
+                throw new IOException(msg);
+            }
+            if (tuple.size()%2!=0) {
+                String msg = "Bin : Some parameters are unmatched.";
+                throw new IOException(msg);
+            }
+        }
+        
+        if (tuple.get(0)==null)
+            return null;
+
+        try {
+            for (int count = 1; count < numParams; count += 2) {
+                if ((count == numParams - 1)
+                        || ((Number)tuple.get(0)).doubleValue() <= ((Number)tuple.get(count + 1)).doubleValue()) {
+
+                    return (String) tuple.get(count);
+                }
+            }
+        } catch (ClassCastException e) {
+            warn("Bin : Data type error", PigWarning.UDF_WARNING_1);
+            return null;
+        } catch (NullPointerException e)
+        {
+            String msg = "Bin : Encounter null in the input";
+            throw new IOException(msg);
+        }
+        String msg = "Bin : Internal Failure";
+        throw new IOException(msg);
+    }
+}

Added: hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/BinCond.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/BinCond.java?rev=799485&view=auto
==============================================================================
--- hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/BinCond.java (added)
+++ hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/BinCond.java Fri Jul 31 01:40:53 2009
@@ -0,0 +1,97 @@
+/*
+ * 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.piggybank.evaluation.decode;
+
+import java.io.IOException;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.PigWarning;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+/**
+* <dl>
+* <dt><b>Syntax:</b></dt>
+* <dd><code>String BinCond(boolean_expression1, mapping_string1, ..., boolean_expressionN, mapping_stringN, other_string)</code>.</dd>
+* <dt><b>Logic:</b></dt>
+* <dd><code>if      (boolean_expression1) return mapping_string1; <br>
+* ...... <br>
+* else if (boolean_expressionN) return mapping_stringN; <br>
+* else                          return other_string; <br>
+* <br></code></dd>
+* </dl>
+*/
+
+public class BinCond extends EvalFunc<String> {
+    int numParams=-1;
+
+    @Override
+    public Schema outputSchema(Schema input) {
+        try {
+            return new Schema(new Schema.FieldSchema(getSchemaName(this
+                    .getClass().getName().toLowerCase(), input),
+                    DataType.CHARARRAY));
+        } catch (Exception e) {
+            return null;
+        }
+    }
+
+    @Override
+    public String exec(Tuple tuple) throws IOException {
+        if (numParams==-1)  // Not initialized
+        {
+            numParams = tuple.size();
+            if (numParams <= 2) {
+                String msg = "BinCond : UDF requires atleast one condition, mapping string & a default string.";
+                throw new IOException(msg);
+            }
+            if (tuple.size()%2!=1) {
+                String msg = "BinCond : Some parameters are unmatched.";
+                throw new IOException(msg);
+            }
+        }
+        
+        if (tuple.get(0)==null)
+            return null;
+        
+        try {
+            for (int count = 0; count < numParams; count += 2) {
+                int returnIndex = -1;
+
+                if (count == numParams - 1) {
+                    returnIndex = count;
+                } else if ((Boolean) tuple.get(count)) {
+                    returnIndex = count + 1;
+                }
+
+                if (returnIndex >= 0) {
+                    return (String) tuple.get(returnIndex);
+                }
+            }
+        } catch (ClassCastException e) {
+            warn("BinCond : Data type error", PigWarning.UDF_WARNING_1);
+            return null;
+        }  catch (NullPointerException e) {
+            String msg = "BinCond : Encounter null in the input";
+            throw new IOException(msg);
+        }
+        String msg = "BinCond : Internal Failure";
+        throw new IOException(msg);
+    }
+}

Added: hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/Decode.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/Decode.java?rev=799485&view=auto
==============================================================================
--- hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/Decode.java (added)
+++ hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/decode/Decode.java Fri Jul 31 01:40:53 2009
@@ -0,0 +1,90 @@
+/*
+ * 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.piggybank.evaluation.decode;
+
+import java.io.IOException;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.PigWarning;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+/**
+* <dl>
+* <dt><b>Syntax:</b></dt>
+* <dd><code>String Decode(expression, value1, mapping_string1, ..., valueN, mapping_stringN, other_string)</code>.</dd>
+* <dt><b>Logic:</b></dt>
+* <dd><code>if      (expression==value1) return mapping_string1; <br>
+* ...... <br>
+* else if (expression==valueN) return mapping_stringN; <br>
+* else                         return other_string;<br></code> <br>
+* expression can be any simple types</dd>
+* </dl>
+*/
+
+public class Decode extends EvalFunc<String> {
+    int numParams = -1;
+    @Override
+    public Schema outputSchema(Schema input) {
+        try {
+            return new Schema(new Schema.FieldSchema(getSchemaName(this
+                    .getClass().getName().toLowerCase(), input),
+                    DataType.CHARARRAY));
+        } catch (Exception e) {
+            return null;
+        }
+    }
+    @Override
+    public String exec(Tuple tuple) throws IOException {
+        if (numParams==-1)  // Not initialized
+        {
+            numParams = tuple.size();
+            if (numParams <= 2) {
+                String msg = "Decode: Atleast an expression and default string is required.";
+                throw new IOException(msg);
+            }
+            if (tuple.size()%2!=0) {
+                String msg = "Decode : Some parameters are unmatched.";
+                throw new IOException(msg);
+            }
+        }
+        
+        if (tuple.get(0)==null)
+            return null;
+
+        try {
+            for (int count = 1; count < numParams - 1; count += 2)
+            {
+                if (tuple.get(count).equals(tuple.get(0)))
+                {
+                    return (String)tuple.get(count+1);
+                }
+            }
+        } catch (ClassCastException e) {
+            warn("Decode : Data type error", PigWarning.UDF_WARNING_1);
+            return null;
+        } catch (NullPointerException e) {
+            String msg = "Decode : Encounter null in the input";
+            throw new IOException(msg);
+        }
+
+        return (String)tuple.get(tuple.size()-1);
+    }
+
+}

Added: hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/HashFNV.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/HashFNV.java?rev=799485&view=auto
==============================================================================
--- hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/HashFNV.java (added)
+++ hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/HashFNV.java Fri Jul 31 01:40:53 2009
@@ -0,0 +1,93 @@
+/*
+ * 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.piggybank.evaluation.string;
+
+import java.io.IOException;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.PigWarning;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+/**
+* <dl>
+* <dt><b>Syntax:</b></dt>
+* <dd><code>long HashFNV(String string_to_hash, [int mod])</code>.</dd>
+* </dl>
+*/
+
+public class HashFNV extends EvalFunc<Long> {
+    static final int FNV1_32_INIT = 33554467;
+    static final int FNV_32_PRIME = 0x01000193;
+    int mMod=-1;
+    public Schema outputSchema(Schema input) {
+        try {
+            return new Schema(new Schema.FieldSchema(getSchemaName(this.getClass().getName().toLowerCase(), input), DataType.LONG));
+        } catch (Exception e) {
+          return null;
+        }
+    }
+
+
+    long hashFnv32Init(int init, String s)
+    {
+        int hval = init;
+
+        byte[] bytes = s.getBytes();
+        for (int i=0;i<bytes.length;i++)
+        {
+            /* multiply by the 32 bit FNV magic prime mod 2^32 */
+            hval *= FNV_32_PRIME;
+            hval ^= bytes[i];
+        }
+        return hval;
+    }
+
+    long hashFnv32(String s)
+    {
+        return hashFnv32Init(FNV1_32_INIT, s);
+    }
+
+    @Override
+    public Long exec(Tuple input) throws IOException {
+        if (input.size()!=1 && input.size()!=2) {
+            String msg = "HashFNV : Only 1 or 2 parameters are allowed.";
+            throw new IOException(msg);
+        }
+        if (input.get(0)==null)
+            return null;
+        if (input.size() == 2)
+        {
+            try {
+                mMod = (Integer)input.get(1);
+            } catch (ClassCastException e) {
+                throw new IOException("HashFNV : 2nd parameter is not Integer",e);
+            }
+        }
+        
+        long v = hashFnv32((String)input.get(0));
+        if (v < 0)
+            v = -v;
+        if (mMod > 0)
+        {
+            v %= mMod;
+        }
+        return v;
+    }
+}

Added: hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/LookupInFiles.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/LookupInFiles.java?rev=799485&view=auto
==============================================================================
--- hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/LookupInFiles.java (added)
+++ hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/LookupInFiles.java Fri Jul 31 01:40:53 2009
@@ -0,0 +1,122 @@
+/*
+ * 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.piggybank.evaluation.string;
+
+import java.io.BufferedReader;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.io.FileLocalizer;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+/**
+* <dl>
+* <dt><b>Syntax:</b></dt>
+* <dd><code>int lookupInFiles(String expression,... <comma separated filelist>)</code>.</dd>
+* <dt><b>Input:</b></dt>
+* <dd><code>files are text files on DFS</code>.</dd>
+* <dt><b>Output:</b></dt>
+* <dd><code>if any file contains expression, return 1, otherwise, 0</code>.</dd>
+* </dl>
+*/
+
+public class LookupInFiles extends EvalFunc<Integer> {
+    boolean initialized = false;
+    ArrayList<String> mFiles = new ArrayList<String>();
+    Map<String, Boolean> mKeys = new HashMap<String, Boolean>();
+    static Map<ArrayList<String>, Map<String, Boolean>> mTables = new HashMap<ArrayList<String>, Map<String, Boolean>>(); 
+
+    @Override
+    public Schema outputSchema(Schema input) {
+      try {
+          return new Schema(new Schema.FieldSchema(getSchemaName(this.getClass().getName().toLowerCase(), input), DataType.INTEGER));
+      } catch (Exception e) {
+        return null;
+      }
+    }
+    
+    public void init(Tuple tuple) throws IOException {
+        for (int count = 1; count < tuple.size(); count++) {
+            if (!(tuple.get(count) instanceof String)) {
+                String msg = "LookupInFiles : Filename should be a string.";
+                throw new IOException(msg);
+            }
+            mFiles.add((String) tuple.get(count));
+        }
+
+        if (mTables.containsKey(mFiles))
+        {
+
+            mKeys = mTables.get(mFiles);
+        }
+        else
+        {
+            Properties props = ConfigurationUtil.toProperties(PigMapReduce.sJobConf);
+            for (int i = 0; i < mFiles.size(); ++i) {
+                // Files contain only 1 column with the key. No Schema. All keys
+                // separated by new line.
+    
+                BufferedReader reader = null;
+    
+                InputStream is = null;
+                try {
+                    is = FileLocalizer.openDFSFile(mFiles.get(i), props);
+                } catch (IOException e) {
+                    String msg = "LookupInFiles : Cannot open file "+mFiles.get(i);
+                    throw new IOException(msg, e);
+                }
+                try {
+                    reader = new BufferedReader(new InputStreamReader(is));
+                    String line;
+                    while ((line = reader.readLine()) != null) {
+                        if (!mKeys.containsKey(line))
+                            mKeys.put(line, true);
+                    }
+                    is.close();
+                } catch (IOException e) {
+                    String msg = "LookupInFiles : Cannot read file "+mFiles.get(i);
+                    throw new IOException(msg, e);
+                }
+            }
+            mTables.put(mFiles, mKeys);
+        }
+        initialized=true;
+    }
+
+    @Override
+    public Integer exec(Tuple input) throws IOException {
+        if (!initialized)
+            init(input);
+        if (input.get(0)==null)
+            return null;
+        if (mKeys.containsKey(input.get(0).toString()))
+            return 1;
+        return 0;
+    }
+}

Added: hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/RegexExtract.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/RegexExtract.java?rev=799485&view=auto
==============================================================================
--- hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/RegexExtract.java (added)
+++ hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/RegexExtract.java Fri Jul 31 01:40:53 2009
@@ -0,0 +1,103 @@
+/*
+ * 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.piggybank.evaluation.string;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.FuncSpec;
+import org.apache.pig.PigWarning;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.FrontendException;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+/**
+* <dl>
+* <dt><b>Syntax:</b></dt>
+* <dd><code>String RegexExtract(String expression, String regex, int match_index)</code>.</dd>
+* <dt><b>Input:</b></dt>
+* <dd><code>expression</code>-<code>source string</code>.</dd>
+* <dd><code>regex</code>-<code>regular expression</code>.</dd>
+* <dd><code>match_index</code>-<code>index of the group to extract</code>.</dd>
+* <dt><b>Output:</b></dt>
+* <dd><code>extracted group, if fail, return null</code>.</dd>
+* </dl>
+*/
+
+public class RegexExtract extends EvalFunc<String> {
+    String mExpression = null;
+    Pattern mPattern = null; 
+    @Override
+    public Schema outputSchema(Schema input) {
+      try {
+          return new Schema(new Schema.FieldSchema(getSchemaName(this.getClass().getName().toLowerCase(), input), DataType.CHARARRAY));
+      } catch (Exception e) {
+        return null;
+      }
+    }
+
+    public String exec(Tuple input) throws IOException {
+        if (input.size()!=3) {
+            String msg = "RegexExtract : Only 3 parameters are allowed.";
+            throw new IOException(msg);
+        }
+        if (input.get(0)==null)
+            return null;
+        try {
+            if (!input.get(1).equals(mExpression))
+            {
+                try
+                {
+                    mExpression = (String)input.get(1);
+                    mPattern = Pattern.compile(mExpression);
+                } catch (Exception e)
+                {
+                    String msg = "RegexExtract : Mal-Formed Regular expression : "+input.get(1);
+                    throw new IOException(msg);
+                }
+            }
+        } catch (NullPointerException e) {
+            String msg = "RegexExtract : Regular expression is null";
+            throw new IOException(msg);
+        }
+        int mIndex = (Integer)input.get(2);
+        
+        Matcher m = mPattern.matcher((String)input.get(0));
+        if (m.find()&&m.groupCount()>=mIndex)
+        {
+            return m.group(mIndex);
+        }
+        warn("RegexExtract : Cannot extract group for input "+input.get(0), PigWarning.UDF_WARNING_1);
+        return null;
+    }
+    @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.INTEGER));
+        funcList.add(new FuncSpec(this.getClass().getName(), s));
+        return funcList;
+    } 
+}

Added: hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/RegexMatch.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/RegexMatch.java?rev=799485&view=auto
==============================================================================
--- hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/RegexMatch.java (added)
+++ hadoop/pig/trunk/contrib/piggybank/java/src/main/java/org/apache/pig/piggybank/evaluation/string/RegexMatch.java Fri Jul 31 01:40:53 2009
@@ -0,0 +1,93 @@
+/*
+ * 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.piggybank.evaluation.string;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Pattern;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.FuncSpec;
+import org.apache.pig.PigWarning;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.FrontendException;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+/**
+* <dl>
+* <dt><b>Syntax:</b></dt>
+* <dd><code>int RegexMatch(String expression, String regex)</code>.</dd>
+* <dt><b>Output:</b></dt>
+* <dd><code>return 1 if expression contains regex, 0 otherwise</code>.</dd>
+* </dl>
+*/
+
+public class RegexMatch extends EvalFunc<Integer> {
+    String mExpression = null;
+    Pattern mPattern = null; 
+    @Override
+    public Schema outputSchema(Schema input) {
+      try {
+          return new Schema(new Schema.FieldSchema(getSchemaName(this.getClass().getName().toLowerCase(), input), DataType.INTEGER));
+      } catch (Exception e) {
+        return null;
+      }
+    }
+    
+    public Integer exec(Tuple input) throws IOException {
+        if (input.size()!=2) {
+            String msg = "RegexMatch : Only 2 parameters are allowed.";
+            throw new IOException(msg);
+        }
+        if (input.get(0)==null)
+            return null;
+        try {
+            if (!input.get(1).equals(mExpression))
+            {
+                mExpression = (String)input.get(1);
+                try
+                {
+                    mPattern = Pattern.compile(mExpression);
+                }
+                catch (Exception e)
+                {
+                    String msg = "RegexMatch : Mal-Formed Regular Expression "+input.get(1);
+                    throw new IOException(msg);
+                } 
+            }
+        } catch (NullPointerException e) {
+            String msg = "RegexMatch : Regular Expression is null ";
+            throw new IOException(msg);
+        }
+
+        if (mPattern.matcher((String)input.get(0)).matches())
+            return 1;
+        return 0;
+    }
+    @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;
+    }
+}

Added: hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/datetime/TestDiffDate.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/datetime/TestDiffDate.java?rev=799485&view=auto
==============================================================================
--- hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/datetime/TestDiffDate.java (added)
+++ hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/datetime/TestDiffDate.java Fri Jul 31 01:40:53 2009
@@ -0,0 +1,59 @@
+/*
+ * 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.piggybank.test.evaluation.datetime;
+
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.pig.piggybank.evaluation.datetime.DiffDate;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class TestDiffDate extends TestCase {
+    @Test
+    public void testDiffDate() throws Exception {
+        Tuple t1 = TupleFactory.getInstance().newTuple(2);
+        t1.set(0, "20090608");
+        t1.set(1, "20090701");
+        
+        Tuple t2 = TupleFactory.getInstance().newTuple(2);
+        t2.set(0, "20090701");
+        t2.set(1, "20080501");
+        
+        Tuple t3 = TupleFactory.getInstance().newTuple(2);
+        t3.set(0, null);
+        t3.set(1, "20090701");
+        
+        Tuple t4 = TupleFactory.getInstance().newTuple(2);
+        t4.set(0, "20090608");
+        t4.set(1, null);
+        
+        DiffDate func = new DiffDate();
+        Integer r = func.exec(t1);
+        assertTrue(r==-23);
+        
+        r = func.exec(t2);
+        assertTrue(r==426);
+        
+        r = func.exec(t3);
+        assertTrue(r==null);
+        
+        r = func.exec(t4);
+        assertTrue(r==null);
+    }
+}

Added: hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/decode/TestDecode.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/decode/TestDecode.java?rev=799485&view=auto
==============================================================================
--- hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/decode/TestDecode.java (added)
+++ hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/decode/TestDecode.java Fri Jul 31 01:40:53 2009
@@ -0,0 +1,180 @@
+/*
+ * 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.piggybank.test.evaluation.decode;
+
+import java.io.IOException;
+
+import junit.framework.TestCase;
+
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.pig.piggybank.evaluation.decode.Bin;
+import org.apache.pig.piggybank.evaluation.decode.BinCond;
+import org.apache.pig.piggybank.evaluation.decode.Decode;
+import org.junit.Test;
+
+public class TestDecode extends TestCase {
+    @Test
+    public void testBin() throws Exception {
+        Tuple t1 = TupleFactory.getInstance().newTuple(8);
+        t1.set(0, new Integer(19));
+        t1.set(1, "infant");
+        t1.set(2, new Integer(5));
+        t1.set(3, "young");
+        t1.set(4, new Integer(20));
+        t1.set(5, "normal");
+        t1.set(6, new Integer(60));
+        t1.set(7, "old");
+        
+        Tuple t2 = TupleFactory.getInstance().newTuple(8);
+        t2.set(0, new Double(1.78));
+        t2.set(1, "S");
+        t2.set(2, new Double(1.70));
+        t2.set(3, "M");
+        t2.set(4, new Double(1.80));
+        t2.set(5, "L");
+        t2.set(6, new Double(1.90));
+        t2.set(7, "XL");
+        
+        Tuple t3 = TupleFactory.getInstance().newTuple(8);
+        t3.set(0, null);
+        t3.set(1, "S");
+        t3.set(2, new Double(1.70));
+        t3.set(3, "M");
+        t3.set(4, new Double(1.80));
+        t3.set(5, "L");
+        t3.set(6, new Double(1.90));
+        t3.set(7, "XL");
+        
+        Tuple t4 = TupleFactory.getInstance().newTuple(8);
+        t4.set(0, new Double(1.78));
+        t4.set(1, null);
+        t4.set(2, new Double(1.70));
+        t3.set(3, "M");
+        t3.set(4, new Double(1.80));
+        t3.set(5, "L");
+        t3.set(6, new Double(1.90));
+        t4.set(7, "XL");
+
+        Bin func = new Bin();
+        String r = func.exec(t1);
+        assertTrue(r.equals("young"));
+        
+        r = func.exec(t2);
+        assertTrue(r.equals("M"));
+        
+        r = func.exec(t3);
+        assertTrue(r==null);
+        
+        try {
+            r = func.exec(t4);
+            fail("Exception not triggered");
+        } catch (IOException e) {
+            assertTrue(e.getMessage().equals("Bin : Encounter null in the input"));
+        }
+    }
+    @Test
+    public void testBinCond() throws Exception {
+        Tuple t1 = TupleFactory.getInstance().newTuple(7);
+        t1.set(0, false);
+        t1.set(1, "s&a");
+        t1.set(2, false);
+        t1.set(3, "a");
+        t1.set(4, true);
+        t1.set(5, "s");
+        t1.set(6, "n");
+        
+        Tuple t2 = TupleFactory.getInstance().newTuple(7);
+        t2.set(0, null);
+        t2.set(1, "s&a");
+        t2.set(2, false);
+        t2.set(3, "a");
+        t2.set(4, true);
+        t2.set(5, "s");
+        t2.set(6, "n");
+
+        Tuple t3 = TupleFactory.getInstance().newTuple(7);
+        t3.set(0, false);
+        t3.set(1, "s&a");
+        t3.set(2, null);
+        t3.set(3, "a");
+        t3.set(4, true);
+        t3.set(5, "s");
+        t3.set(6, "n");
+        
+        BinCond func = new BinCond();
+        String r = func.exec(t1);
+        assertTrue(r.equals("s"));
+        r = func.exec(t2);
+        assertTrue(r==null);
+        try {
+            r = func.exec(t3);
+            fail("Exception not triggered");
+        } catch (IOException e) {
+            assertTrue(e.getMessage().equals("BinCond : Encounter null in the input"));
+        }
+    }
+    @Test
+    public void testDecode() throws Exception {
+        Tuple t1 = TupleFactory.getInstance().newTuple(6);
+        t1.set(0, new Integer(1));
+        t1.set(1, 0);
+        t1.set(2, "Sales");
+        t1.set(3, 1);
+        t1.set(4, "Engineering");
+        t1.set(5, "Other");
+        
+        Tuple t2 = TupleFactory.getInstance().newTuple(6);
+        t2.set(0, new Integer(3));
+        t2.set(1, 0);
+        t2.set(2, "Sales");
+        t2.set(3, 1);
+        t2.set(4, "Engineering");
+        t2.set(5, "Other");
+
+        Tuple t3 = TupleFactory.getInstance().newTuple(6);
+        t3.set(0, null);
+        t3.set(1, 0);
+        t3.set(2, "Sales");
+        t3.set(3, 1);
+        t3.set(4, "Engineering");
+        t3.set(5, "Other");
+
+        Tuple t4 = TupleFactory.getInstance().newTuple(6);
+        t4.set(0, new Integer(1));
+        t4.set(1, null);
+        t4.set(2, "Sales");
+        t4.set(3, 1);
+        t4.set(4, "Engineering");
+        t4.set(5, "Other");
+        
+        Decode func = new Decode();
+        String r = func.exec(t1);
+        assertTrue(r.equals("Engineering"));        
+        r = func.exec(t2);
+        assertTrue(r.equals("Other"));
+        r = func.exec(t3);
+        assertTrue(r==null);
+        try {
+            r = func.exec(t4);
+            fail("Exception not triggered");
+        } catch (IOException e) {
+            assertTrue(e.getMessage().equals("Decode : Encounter null in the input"));
+        }
+    }
+}

Added: hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestHashFNV.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestHashFNV.java?rev=799485&view=auto
==============================================================================
--- hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestHashFNV.java (added)
+++ hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestHashFNV.java Fri Jul 31 01:40:53 2009
@@ -0,0 +1,50 @@
+/*
+ * 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.piggybank.test.evaluation.string;
+
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.pig.piggybank.evaluation.string.HashFNV;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class TestHashFNV extends TestCase {
+    @Test
+    public void testHashFNV() throws Exception {
+        Tuple t1 = TupleFactory.getInstance().newTuple(2);
+        t1.set(0, "0000000000065&f=a&br=65");
+        t1.set(1, 10000);
+        
+        Tuple t2 = TupleFactory.getInstance().newTuple(2);
+        t2.set(0, "024ulhl0dq1tl&b=2");
+        t2.set(1, 100);
+        
+        Tuple t3 = TupleFactory.getInstance().newTuple(2);
+        t3.set(0, null);
+        t3.set(1, 100);
+        
+        HashFNV func = new HashFNV();
+        Long r = func.exec(t1);
+        assertTrue(r==6228);
+        r = func.exec(t2);
+        assertTrue(r==31);
+        r = func.exec(t3);
+        assertTrue(r==null);
+    }
+}

Added: hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestLookupInFiles.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestLookupInFiles.java?rev=799485&view=auto
==============================================================================
--- hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestLookupInFiles.java (added)
+++ hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestLookupInFiles.java Fri Jul 31 01:40:53 2009
@@ -0,0 +1,82 @@
+/*
+ * 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.piggybank.test.evaluation.string;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.PrintStream;
+import java.util.Iterator;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.pig.ExecType;
+import org.apache.pig.PigServer;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.test.MiniCluster;
+import org.apache.pig.test.Util;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class TestLookupInFiles extends TestCase {
+    MiniCluster cluster = MiniCluster.buildCluster();
+    private PigServer pigServer;
+   
+    public void setUp() throws Exception{
+        pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+    }
+    @Test
+    public void testLookupInFiles() throws Exception {
+        File tmpFile = File.createTempFile("test", ".txt");
+        PrintStream ps1 = new PrintStream(new FileOutputStream(tmpFile));
+        
+        ps1.println("one");
+        ps1.println("notexist");
+        ps1.println("three");
+        ps1.close();
+        
+        File lookupFile1 = File.createTempFile("lookup", ".txt");
+        PrintStream lps1 = new PrintStream(new FileOutputStream(lookupFile1));
+        
+        lps1.println("one");
+        lps1.println("two");
+        lps1.println("three");
+        lps1.close();
+        
+        File lookupFile2 = File.createTempFile("lookup", "txt");
+        PrintStream lps2 = new PrintStream(new FileOutputStream(lookupFile2));
+        
+        lps2.println("one");
+        lps2.println("ten");
+        lps2.println("eleven");
+        lps2.close();
+        
+        FileSystem fs = FileSystem.get(ConfigurationUtil.toConfiguration(pigServer.getPigContext().getProperties()));
+        fs.copyFromLocalFile(new Path(lookupFile1.toString()), new Path("lookup1"));
+        fs.copyFromLocalFile(new Path(lookupFile1.toString()), new Path("lookup2"));
+        pigServer.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "' AS (key:chararray);");
+        pigServer.registerQuery("B = FOREACH A GENERATE org.apache.pig.piggybank.evaluation.string.LookupInFiles(key, 'lookup1', 'lookup2');");
+        Iterator<Tuple> iter = pigServer.openIterator("B");
+        
+        int r = (Integer)iter.next().get(0);
+        assertTrue(r==1);
+        r = (Integer)iter.next().get(0);
+        assertTrue(r==0);
+    }
+}

Added: hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestRegex.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestRegex.java?rev=799485&view=auto
==============================================================================
--- hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestRegex.java (added)
+++ hadoop/pig/trunk/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestRegex.java Fri Jul 31 01:40:53 2009
@@ -0,0 +1,77 @@
+/*
+ * 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.piggybank.test.evaluation.string;
+
+import junit.framework.TestCase;
+
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.pig.piggybank.evaluation.string.RegexExtract;
+import org.apache.pig.piggybank.evaluation.string.RegexMatch;
+import org.junit.Test;
+
+public class TestRegex extends TestCase {
+    @Test
+    public void testRegexMatch() throws Exception {
+        Tuple t1 = TupleFactory.getInstance().newTuple(2);
+        t1.set(0, "/search/iy/xxx");
+        t1.set(1, "^\\/search\\/iy\\/.*");
+        
+        Tuple t2 = TupleFactory.getInstance().newTuple(2);
+        t2.set(0, "http://yahoo.com");
+        t2.set(1, "^\\/search\\/iy\\/.*");
+
+        Tuple t3 = TupleFactory.getInstance().newTuple(2);
+        t3.set(0, null);
+        t3.set(1, "^\\/search\\/iy\\/.*");
+        
+        RegexMatch func = new RegexMatch();
+        Integer r = func.exec(t1);
+        assertTrue(r==1);
+        r = func.exec(t2);
+        assertTrue(r==0);
+        r = func.exec(t3);
+        assertTrue(r==null);
+    }
+    @Test
+    public void testRegexExtract() throws Exception {
+        Tuple t1 = TupleFactory.getInstance().newTuple(3);
+        t1.set(0, "/search/iy/term1/test");
+        t1.set(1, "^\\/search\\/iy\\/(.*?)\\/.*");
+        t1.set(2, 1);
+        
+        Tuple t2 = TupleFactory.getInstance().newTuple(3);
+        t2.set(0, "/search/iy/term1/test");
+        t2.set(1, "^\\/search\\/iy\\/(.*?)\\/.*");
+        t2.set(2, 2);
+        
+        Tuple t3 = TupleFactory.getInstance().newTuple(3);
+        t3.set(0, null);
+        t3.set(1, "^\\/search\\/iy\\/(.*?)\\/.*");
+        t3.set(2, 2);
+
+        
+        RegexExtract func = new RegexExtract();
+        String r = func.exec(t1);
+        assertTrue(r.equals("term1"));
+        r = func.exec(t2);
+        assertTrue(r==null);
+        r = func.exec(t3);
+        assertTrue(r==null);
+    }
+}