You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by th...@apache.org on 2010/08/13 22:38:41 UTC

svn commit: r985354 - in /hadoop/pig/trunk: ./ lib/ src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/ src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/ src/org/apache/pig/backend/hadoop/executionengine/...

Author: thejas
Date: Fri Aug 13 20:38:40 2010
New Revision: 985354

URL: http://svn.apache.org/viewvc?rev=985354&view=rev
Log:
PIG-965: PERFORMANCE: optimize common case in matches (PORegex) (ankit.modi via olgan)

Added:
    hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/
    hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/CompiledAutomaton.java
    hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/CompiledRegex.java
    hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/NonConstantRegex.java
    hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/RegexImpl.java
    hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/RegexInit.java
    hadoop/pig/trunk/test/org/apache/pig/test/TestPORegexp.java
Modified:
    hadoop/pig/trunk/CHANGES.txt
    hadoop/pig/trunk/build.xml
    hadoop/pig/trunk/lib/automaton.jar
    hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogToPhyTranslationVisitor.java
    hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/PORegexp.java
    hadoop/pig/trunk/src/org/apache/pig/impl/util/JarManager.java

Modified: hadoop/pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=985354&r1=985353&r2=985354&view=diff
==============================================================================
--- hadoop/pig/trunk/CHANGES.txt (original)
+++ hadoop/pig/trunk/CHANGES.txt Fri Aug 13 20:38:40 2010
@@ -26,6 +26,9 @@ PIG-1249: Safe-guards against misconfigu
 
 IMPROVEMENTS
 
+PIG-965: PERFORMANCE: optimize common case in matches (PORegex) (ankit.modi
+via olgan)
+
 PIG-103: Shared Job /tmp location should be configurable (niraj via rding)
 
 PIG-1496: Mandatory rule ImplicitSplitInserter (yanz via daijy)
@@ -363,9 +366,6 @@ PIG-1156: Add aliases to ExecJobs and Ph
 
 PIG-1161: add missing license headers (dvryaboy via olgan)
 
-PIG-965: PERFORMANCE: optimize common case in matches (PORegex) (ankit.modi
-via olgan)
-
 PIG-760:  Add a new PigStorageSchema load/store function that 
           store schemas for text files (dvryaboy via gates)
 
@@ -548,6 +548,9 @@ comments (thejas via olgan)
 PIG-1064: Behaviour of COGROUP with and without schema when using "*" operator
 (pradeepkth)
 
+PIG-965: PERFORMANCE: optimize common case in matches (PORegex) (ankit.modi
+via )
+
 PIG-1086: Nested sort by * throw exception (rding via daijy)
 
 PIG-1146: Inconsistent column pruning in LOUnion (daijy)

Modified: hadoop/pig/trunk/build.xml
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/build.xml?rev=985354&r1=985353&r2=985354&view=diff
==============================================================================
--- hadoop/pig/trunk/build.xml (original)
+++ hadoop/pig/trunk/build.xml Fri Aug 13 20:38:40 2010
@@ -48,6 +48,7 @@
     <property name="build.encoding" value="UTF8" />
     <!-- TODO with only one version of hadoop in the lib folder we do not need that anymore -->
     <property name="hadoop.jarfile" value="hadoop20.jar" />
+    <property name="automaton.jarfile" value="automaton.jar" />
     <property name="hbase.jarfile" value="hbase-0.20.0.jar" />
     <property name="hbase.test.jarfile" value="hbase-0.20.0-test.jar" />
 	<property name="zookeeper.jarfile" value="zookeeper-hbase-1329.jar" />
@@ -168,6 +169,7 @@
     <path id="classpath">
 	<path refid="compile.classpath"/>	
         <fileset file="${lib.dir}/${hadoop.jarfile}" />
+        <fileset file="${lib.dir}/${automaton.jarfile}" />
         <fileset file="${lib.dir}/${hbase.jarfile}" />
         <fileset file="${lib.dir}/${hbase.test.jarfile}" />
     	<fileset file="${lib.dir}/${zookeeper.jarfile}"/>
@@ -464,6 +466,7 @@
                 </section>
             </manifest>
             <zipfileset src="${lib.dir}/${hadoop.jarfile}" />
+            <zipfileset src="${lib.dir}/${automaton.jarfile}" />
             <zipfileset src="${ivy.lib.dir}/junit-${junit.version}.jar" />
             <zipfileset src="${ivy.lib.dir}/jsch-${jsch.version}.jar" />
             <zipfileset src="${ivy.lib.dir}/jline-${jline.version}.jar" />

Modified: hadoop/pig/trunk/lib/automaton.jar
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/lib/automaton.jar?rev=985354&r1=985353&r2=985354&view=diff
==============================================================================
Files hadoop/pig/trunk/lib/automaton.jar (original) and hadoop/pig/trunk/lib/automaton.jar Fri Aug 13 20:38:40 2010 differ

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogToPhyTranslationVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogToPhyTranslationVisitor.java?rev=985354&r1=985353&r2=985354&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogToPhyTranslationVisitor.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogToPhyTranslationVisitor.java Fri Aug 13 20:38:40 2010
@@ -298,8 +298,16 @@ public class LogToPhyTranslationVisitor 
         List<LogicalOperator> predecessors = lp.getPredecessors(op);
         if (predecessors == null)
             return;
-        for (LogicalOperator lo : predecessors) {
+        int counter = 0;
+        for (LogicalOperator lo : predecessors) {         
             PhysicalOperator from = logToPhyMap.get(lo);
+            // If the source is a ConstantExpression we notify PORegexp about it.
+            // It helps to optimize regex operation
+            if( from.getClass().getCanonicalName().compareTo(ConstantExpression.class.getCanonicalName()) == 0
+                    && counter == 1 ) {
+                ((PORegexp)exprOp).setConstExpr(true);
+            }
+            counter++;
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/PORegexp.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/PORegexp.java?rev=985354&r1=985353&r2=985354&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/PORegexp.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/PORegexp.java Fri Aug 13 20:38:40 2010
@@ -17,11 +17,11 @@
  */
 package org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators;
 
-import java.util.regex.PatternSyntaxException;
-
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.regex.RegexInit;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.regex.RegexImpl;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.data.DataType;
 import org.apache.pig.impl.plan.OperatorKey;
@@ -30,10 +30,9 @@ import org.apache.pig.impl.plan.VisitorE
 
 public class PORegexp extends BinaryComparisonOperator {
 
-    /**
-     * 
-     */
     private static final long serialVersionUID = 1L;
+    
+    private RegexImpl impl = null;
 
     public PORegexp(OperatorKey k) {
         this(k, -1);
@@ -42,6 +41,14 @@ public class PORegexp extends BinaryComp
     public PORegexp(OperatorKey k, int rp) {
         super(k, rp);
         resultType = DataType.BOOLEAN;
+        // We set impl RegexInit.
+        // RegexInit decides what plan to choose after looking
+        // at first tuple. And resets this.impl to new implementation
+        this.impl = new RegexInit(this);
+    }
+    
+    public void setImplementation( RegexImpl impl ) {
+        this.impl = impl;
     }
 
     @Override
@@ -53,6 +60,10 @@ public class PORegexp extends BinaryComp
     public String name() {
         return "Matches - " + mKey.toString();
     }
+    
+    public void setConstExpr( boolean rhsConstant ) {
+        ((RegexInit)this.impl).setConstExpr(rhsConstant);
+    }
 
     @Override
     public Result getNext(Boolean bool) throws ExecException {
@@ -61,7 +72,6 @@ public class PORegexp extends BinaryComp
             return r;
         }
         
-        byte status;
         Result left, right;
 
         left = lhs.getNext(dummyString);
@@ -70,13 +80,14 @@ public class PORegexp extends BinaryComp
         if (trueRef == null) initializeRefs();
         if (left.returnStatus != POStatus.STATUS_OK || left.result == null) return left;
         if (right.returnStatus != POStatus.STATUS_OK || right.result == null) return right;
-        if (((String)left.result).matches((String)right.result)) {
+        
+        if( impl.match((String)(left.result),(String)(right.result)) ) {
             left.result = trueRef;
         } else {
             left.result = falseRef;
         }
         return left;
-    }
+    }    
 
     @Override
     public PORegexp clone() throws CloneNotSupportedException {

Added: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/CompiledAutomaton.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/CompiledAutomaton.java?rev=985354&view=auto
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/CompiledAutomaton.java (added)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/CompiledAutomaton.java Fri Aug 13 20:38:40 2010
@@ -0,0 +1,39 @@
+/*
+ * 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.backend.hadoop.executionengine.physicalLayer.expressionOperators.regex;
+
+import dk.brics.automaton.Automaton;
+import dk.brics.automaton.RegExp;
+import dk.brics.automaton.RunAutomaton;
+
+public class CompiledAutomaton implements RegexImpl {
+
+    private RunAutomaton runauto = null;
+    
+    public CompiledAutomaton( String rhsPattern ) {
+        RegExp regexpr = new dk.brics.automaton.RegExp(rhsPattern);
+        Automaton auto = regexpr.toAutomaton();
+        this.runauto = new RunAutomaton(auto, true);
+    }
+    
+    @Override
+    public boolean match(String lhs, String rhs) {
+        return this.runauto.run(lhs);
+    }
+
+}

Added: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/CompiledRegex.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/CompiledRegex.java?rev=985354&view=auto
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/CompiledRegex.java (added)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/CompiledRegex.java Fri Aug 13 20:38:40 2010
@@ -0,0 +1,37 @@
+/*
+ * 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.backend.hadoop.executionengine.physicalLayer.expressionOperators.regex;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class CompiledRegex implements RegexImpl {
+
+    private Matcher m = null;
+    
+    public CompiledRegex( Pattern rhsPattern ) {
+        this.m = rhsPattern.matcher("");
+    }
+    
+    @Override
+    public boolean match(String lhs, String rhs) {
+        m.reset(lhs);            
+        return m.matches();
+    }
+
+}

Added: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/NonConstantRegex.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/NonConstantRegex.java?rev=985354&view=auto
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/NonConstantRegex.java (added)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/NonConstantRegex.java Fri Aug 13 20:38:40 2010
@@ -0,0 +1,49 @@
+/*
+ * 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.backend.hadoop.executionengine.physicalLayer.expressionOperators.regex;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class NonConstantRegex implements RegexImpl {
+
+    private Pattern pattern = null;
+    
+    private String oldString = null;
+    
+    private Matcher matcher = null;
+
+    @Override
+    public boolean match(String lhs, String rhs) {
+        // We first check for length so the comparison is faster
+        // and then we directly check for difference.
+        // I havent used equals as first two comparisons,
+        // same Object and isInstanceOf does not apply in this case.
+        if( oldString == null
+                || rhs.length() != oldString.length() 
+                || rhs.compareTo(oldString) != 0 ) {
+            oldString = rhs;
+            pattern = Pattern.compile(oldString);
+            matcher = pattern.matcher(lhs);
+        } else {
+            matcher.reset( lhs );
+        }
+        return matcher.matches();
+    }
+
+}

Added: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/RegexImpl.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/RegexImpl.java?rev=985354&view=auto
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/RegexImpl.java (added)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/RegexImpl.java Fri Aug 13 20:38:40 2010
@@ -0,0 +1,23 @@
+/*
+ * 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.backend.hadoop.executionengine.physicalLayer.expressionOperators.regex;
+
+// General interface for regexComparison
+public interface RegexImpl {
+    boolean match( String lhs, String rhs );
+}

Added: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/RegexInit.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/RegexInit.java?rev=985354&view=auto
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/RegexInit.java (added)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/regex/RegexInit.java Fri Aug 13 20:38:40 2010
@@ -0,0 +1,215 @@
+/*
+ * 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.backend.hadoop.executionengine.physicalLayer.expressionOperators.regex;
+
+import java.io.Serializable;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.PORegexp;
+
+public class RegexInit implements RegexImpl, Serializable {
+
+    private static final long serialVersionUID = 1L;
+    // Intersection and subtraction ( subtraction cannot be used w/o intersection )
+    // ,reluctant and possesive quantifiers 
+    // is only possible in java.util.regex
+    private static final String[] javaRegexOnly = { "&&", "??", "*?", "+?", "}?",
+            "?+", "*+", "++", "}+", "^", "$", "(?" };
+
+    PORegexp regexop = null;
+    int side = -1;
+    boolean rhsConstant = false;
+    
+    public RegexInit(PORegexp regexoperator ) {
+        this.regexop = regexoperator;
+    }
+    
+    public void setConstExpr( boolean rhsConstant ) {
+        this.rhsConstant = rhsConstant;
+    }
+  
+    /**
+     * This function determines the type of pattern we are working with
+     * The return value of the function determines the type we are expecting
+     * @param pattern
+     * @return int, 0 means this is java.util.regex,
+     * 1 means this is dk.brics.automaton
+     */
+    private int determineBestRegexMethod( String pattern ) {
+
+        for( int i = 0; i < javaRegexOnly.length; i++ ) {
+            for( int j = pattern.length(); j > 0; ) {
+                j = pattern.lastIndexOf(javaRegexOnly[i], j );
+                if( j > 0 ) {
+                    int precedingEsc = precedingEscapes(pattern, j);
+                    if( precedingEsc %2 == 0 ) {
+                        return 0;
+                    }
+                    j = j - precedingEsc;
+                } else if ( j == 0 ) {
+                    return 0;
+                }
+            }
+        }
+
+        // Determine if there are any complex unions in pattern
+        // Complex unions are [a-m[n-z]]
+        int index = pattern.indexOf('[');
+        if( index >= 0 ) {
+            int precedingEsc = precedingEscapes(pattern, index);
+            if( index != 0 ) {
+                while( precedingEsc %2 == 1 ) {
+                    index = pattern.indexOf('[', index + 1);
+                    precedingEsc = precedingEscapes(pattern, index);
+                }
+            }
+            int index2 = 0;
+            int index3 = 0;
+            while( index != -1 && index < pattern.length() ) {
+                index2 = pattern.indexOf(']', index );
+                if( index2 == -1 ) {
+                    break;
+                }
+                precedingEsc = precedingEscapes(pattern, index2);
+                // Find the next ']' which is not '\\]'
+                while( precedingEsc %2 == 1 ) {
+                    index2 = pattern.indexOf(']', index2 + 1);
+                    precedingEsc = precedingEscapes(pattern, index2);
+                }                
+                if( index2 == -1 ) {
+                    break;
+                }
+                index3 = pattern.indexOf('[', index + 1 );
+                precedingEsc = precedingEscapes(pattern, index3);
+                if( index3 == -1 ) {
+                    break;
+                }
+                // Find the next '[' which is not '\\['
+                while( precedingEsc %2 == 1 ) {
+                    index3 = pattern.indexOf('[', index3 + 1);
+                    precedingEsc = precedingEscapes(pattern, index3);
+                }
+                if( index3 == -1 ) {
+                    break;
+                }
+                if( index3 < index2 ) {
+                    return 0;
+                }
+                index = index3;
+            }
+        }
+
+        index = pattern.lastIndexOf('\\');
+        if( index > -1 ) {
+            int precedingEsc = precedingEscapes(pattern, index);
+            // This is the case where we have complex regexes
+            // e.g. \d, \D, \s...etc
+            while( index != -1 ) {
+                if( precedingEsc %2 == 0 && (index + 1 ) < pattern.length() ) {
+                    char index_1 = pattern.charAt(index + 1 );
+                    if( index_1 == '1' || index_1 == '2' ||
+                            index_1 == '3' || index_1 == '4' ||
+                            index_1 == '5' || index_1 == '6' ||
+                            index_1 == '7' || index_1 == '8' ||
+                            index_1 == '9' ||
+                            index_1 == 'a' || index_1 == 'e' ||
+                            index_1 == '0' || index_1 == 'x' ||
+                            index_1 == 'u' ||
+                            index_1 == 'c' ||
+                            index_1 == 'Q' ||
+                            index_1 == 'w' || index_1 == 'W' ||
+                            index_1 == 'd' || index_1 == 'D' ||
+                            index_1 == 's' || index_1 == 'S' ||
+                            index_1 == 'p' || index_1 == 'P' ||
+                            index_1 == 'b' || index_1 == 'B' ||
+                            index_1 == 'A' || index_1 == 'G' ||
+                            index_1 == 'z' || index_1 == 'Z'   
+                    ) {
+                        return 0; 
+                    }
+                }
+
+                // We skip past all the escapes
+                index = index - ( precedingEsc + 1 );
+                precedingEsc = -1;
+                if( index >= 0 ){
+                    index = pattern.lastIndexOf('\\',index);
+                    precedingEsc = precedingEscapes(pattern, index);
+                }
+            }
+        }
+        return 1;
+    }    
+
+    private int precedingEscapes( String pattern, int startIndex ) {
+        if( startIndex > 0 ) {
+            // This is the case when there are an odd number of escapes '//'
+            int precedingEscapes = 0;
+            for(int j = startIndex - 1; j >= 0; j-- ) {
+                if( pattern.charAt(j) == '\\' ) {
+                    precedingEscapes++;
+                } else {
+                    break;
+                }
+            }
+            return precedingEscapes;
+        } else if ( startIndex == 0 ) {
+            return 0;
+        }
+        return -1;
+    }
+
+    private RegexImpl compile( String pattern ) {
+        RegexImpl impl = null;
+        int regexMethod = determineBestRegexMethod(pattern);
+        switch( regexMethod ) {
+        case 0:
+            impl = new CompiledRegex(Pattern.compile(pattern));
+            break;
+        case 1:
+            try {
+                impl = new CompiledAutomaton(pattern);
+            } catch( IllegalArgumentException e ) {
+                Log log = LogFactory.getLog(getClass());
+                log.debug("Got an IllegalArgumentException for Pattern: " + pattern );
+                log.debug(e.getMessage());
+                log.debug("Switching to java.util.regex" );
+                impl = new CompiledRegex(Pattern.compile(pattern));
+            }
+            break;
+        default:
+            break;
+        }
+        return impl;
+    }
+
+    @Override
+    public boolean match(String lhs, String rhs) {
+        RegexImpl impl = null;
+        if( rhsConstant ) {
+            impl = compile( rhs );
+        } else {
+            impl = new NonConstantRegex();
+        }
+        this.regexop.setImplementation( impl );
+        return impl.match(lhs, rhs);
+    }      
+
+}

Modified: hadoop/pig/trunk/src/org/apache/pig/impl/util/JarManager.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/util/JarManager.java?rev=985354&r1=985353&r2=985354&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/impl/util/JarManager.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/impl/util/JarManager.java Fri Aug 13 20:38:40 2010
@@ -84,7 +84,8 @@ public class JarManager {
         }
     }
 
-    final static String pigPackagesToSend[] = { "org/apache/pig","org/apache/tools/bzip2r" };
+    final static String pigPackagesToSend[] = { "org/apache/pig","org/apache/tools/bzip2r",
+        "dk/brics/automaton" };
     
     /**
      * Create a jarfile in a temporary path, that is a merge of all the jarfiles containing the

Added: hadoop/pig/trunk/test/org/apache/pig/test/TestPORegexp.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestPORegexp.java?rev=985354&view=auto
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestPORegexp.java (added)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestPORegexp.java Fri Aug 13 20:38:40 2010
@@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.test;
+
+import java.lang.reflect.Method;
+import java.util.Random;
+import java.util.regex.Pattern;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.PORegexp;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.regex.NonConstantRegex;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.regex.CompiledAutomaton;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.regex.CompiledRegex;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.regex.RegexInit;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.junit.Before;
+import org.junit.Test;
+import junit.framework.TestCase;
+
+public class TestPORegexp extends TestCase {
+
+    static Random r = new Random();
+    
+    @Before
+    @Override
+    protected void setUp() throws Exception {
+        
+    }
+    
+    @Test
+    public void testOrdering() {
+        try {
+            CompiledAutomaton auto2 = new CompiledAutomaton("[a-z]{3}");
+            assertEquals(false, auto2.match("1234", "abc") );
+            assertEquals(true, auto2.match("abc", "1234") );
+ 
+            CompiledRegex regex2 = new CompiledRegex(Pattern.compile("[a-z]{3}"));
+            assertEquals(false, regex2.match("1234", "abc") );
+            assertEquals(true, regex2.match("abc", "1234") );
+            
+            NonConstantRegex ncr = new NonConstantRegex();
+            assertEquals(false, ncr.match("1234", "abc"));
+            assertEquals(false, ncr.match("abc", "1234"));
+            assertEquals(true, ncr.match("1234", "\\d\\d\\d\\d"));
+            assertEquals(true, ncr.match("abc", "[a-z]{3}"));
+            
+        } catch( Exception e ){ 
+            fail();
+        }
+    }
+    
+    @Test
+    public void testRegexDetermination() {
+        try {
+            Method m = RegexInit.class.getDeclaredMethod
+            ("determineBestRegexMethod", String.class);
+            m.setAccessible(true);
+            
+            RegexInit regex = new RegexInit(new PORegexp(new OperatorKey()));
+            
+            assertEquals(1, m.invoke(regex, "abc") );
+            
+            assertEquals(1, m.invoke(regex, "\\\\abc") );
+            
+            assertEquals(1,  m.invoke(regex, "abc.*"));
+            
+            assertEquals(1,  m.invoke(regex, ".*abc"));
+            
+            assertEquals(1,  m.invoke(regex, ".*abc.*"));
+            
+            assertEquals(1,  m.invoke(regex, ".*abc\\.*"));
+            
+            assertEquals(1,  m.invoke(regex, ".*abc\\\\"));
+            
+            assertEquals(0,  m.invoke(regex, ".*abc\\d"));
+            
+            assertEquals(0,  m.invoke(regex, ".*abc\\s"));
+            
+            assertEquals(0,  m.invoke(regex, ".*abc\\Sw"));
+            
+            assertEquals(0,  m.invoke(regex, "abc\\Sw"));
+
+            assertEquals(0,  m.invoke(regex, "a\\Q"));
+            
+            assertEquals(0,  m.invoke(regex, "\\QThis is something"));
+            
+            assertEquals(0,  m.invoke(regex, "(\\w)*\\s\\1"));
+
+            assertEquals(0,  m.invoke(regex, "[^a]bc"));
+            
+            assertEquals(0,  m.invoke(regex, "\\p{Alpha}hi"));
+            
+            assertEquals(0,  m.invoke(regex, "\\d{1,2}hi"));
+            
+            assertEquals(0,  m.invoke(regex, "^abc.*"));
+            
+            assertEquals(1,  m.invoke(regex, ".*[A-F]{2,3}.*"));            
+            
+            assertEquals(0,  m.invoke(regex, "\\d+"));
+            
+            assertEquals(0,  m.invoke(regex, "\\d{2,3}"));
+            
+            assertEquals(0,  m.invoke(regex, "\\\\\\d{2,3}"));
+            
+            assertEquals(0,  m.invoke(regex, ".*\\d{2,3}.*"));
+            
+            assertEquals(0,  m.invoke(regex, "\\d\\.0\\d"));
+            
+            assertEquals(0,  m.invoke(regex, "[^f]ed.*"));
+            
+            assertEquals(0,  m.invoke(regex, "[a-m[n-z]]"));
+            
+            assertEquals(0,  m.invoke(regex, "[a-z&&[def]]"));
+
+            assertEquals(0,  m.invoke(regex, "[a-z&&[^abc]]"));
+            
+            assertEquals(1,  m.invoke(regex, "[a-m\\[n-z\\]"));
+
+            assertEquals(1,  m.invoke(regex, "[a-m\\\\\\[n-z\\\\\\]]"));
+
+            assertEquals(0,  m.invoke(regex, "[a-m\\\\\\[n-z\\\\\\][0-9]]"));
+            
+            assertEquals(0,  m.invoke(regex, "[a-m\\\\[n-z]]"));
+            
+            assertEquals(0,  m.invoke(regex, "\\\\\\[[a-m\\\\\\[n-z\\\\\\][0-9]]"));
+            
+            assertEquals(0,  m.invoke(regex, "[a-z]??" ));
+            
+            assertEquals(0,  m.invoke(regex, "[a-z]*?" ));
+
+            assertEquals(0,  m.invoke(regex, "[a-z]+?" ));
+            
+            assertEquals(0,  m.invoke(regex, "[a-z]{4}?" ));
+
+            assertEquals(0,  m.invoke(regex, "[a-z]{2,4}?" ));
+            
+            assertEquals(1,  m.invoke(regex, "[a-z]\\??" ));
+            
+            assertEquals(1,  m.invoke(regex, "[a-z]\\*?" ));
+
+            assertEquals(1,  m.invoke(regex, "[a-z]\\+?" ));
+            
+            assertEquals(1,  m.invoke(regex, "[a-z]{4\\}?" ));
+
+            assertEquals(1,  m.invoke(regex, "[a-z]{2,4\\}?" ));
+            
+            assertEquals(0,  m.invoke(regex, "[a-z]?+" ));
+            
+            assertEquals(0,  m.invoke(regex, "[a-z]*+" ));
+
+            assertEquals(0,  m.invoke(regex, "[a-z]++" ));
+            
+            assertEquals(0,  m.invoke(regex, "[a-z]{4}+" ));
+
+            assertEquals(0,  m.invoke(regex, "[a-z]{2,4}+" ));
+            
+            assertEquals(1,  m.invoke(regex, "[a-z]\\?+" ));
+            
+            assertEquals(1,  m.invoke(regex, "[a-z]\\*+" ));
+
+            assertEquals(1,  m.invoke(regex, "[a-z]\\++" ));
+            
+            assertEquals(1,  m.invoke(regex, "[a-z]{4\\}+" ));
+
+            assertEquals(1,  m.invoke(regex, "[a-z]{2,4\\}+" ));
+
+            assertEquals(1,  m.invoke(regex, "[a-m\\[n-z\\]]" ));
+            
+            assertEquals(0,  m.invoke(regex, "\\0101" ));
+            
+            assertEquals(0,  m.invoke(regex, "\\x0A" ));
+
+            assertEquals(0,  m.invoke(regex, "\\u000A" ));
+
+            assertEquals(0,  m.invoke(regex, "&&" ));
+
+            assertEquals(1,  m.invoke(regex, "\\&&asdkfjalsdf" ));
+
+            assertEquals(0,  m.invoke(regex, "&&asdf\\&&" ));
+
+            assertEquals(0,  m.invoke(regex, "&&asdf\\&&asdfasdf" ));
+
+            assertEquals(0,  m.invoke(regex, "&&asdfas\\&&asdfasdfa\\&&" ));
+
+            assertEquals(0,  m.invoke(regex, "&&asdflj&&" ));
+
+            assertEquals(0,  m.invoke(regex, "\\\\&&asdfasdf" ));
+
+            assertEquals(1,  m.invoke(regex, "\\\\\\&&asdfasdf" ));
+            
+            assertEquals(0,  m.invoke(regex, "\\\\&&asdfasdf&&" ));
+
+            assertEquals(0,  m.invoke(regex, "\\&&asdfasdf\\\\&&" ));
+
+            assertEquals(0,  m.invoke(regex, "\\&&asd&&fasdf\\\\\\&&" ));
+            
+            assertEquals(0,  m.invoke(regex, "\\dasdfasdf" ));
+
+            assertEquals(1,  m.invoke(regex, "\\\\dasdfasdf" ));
+
+            assertEquals(0,  m.invoke(regex, "\\\\dasdfasdf\\d" ));
+
+            assertEquals(0,  m.invoke(regex, "\\\\dasdf\\dasdf\\\\d" ));
+
+            assertEquals(0,  m.invoke(regex, "\\\\dasd\\\\dfasdf\\d" ));
+
+            assertEquals(1,  m.invoke(regex, "\\\\dasdfasdf\\" ));
+
+            assertEquals(0,  m.invoke(regex, "\\dasase\\\\dfasdf\\" ));
+
+            assertEquals(1,  m.invoke(regex, "\\\\dasdfasdf\\\\" ));
+
+
+
+        } catch( Exception e ) {
+            System.err.println(e.getMessage());
+            fail();
+        }
+    }
+}