You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2015/04/16 19:12:04 UTC

[1/2] phoenix git commit: PHOENIX-1876 Check for null function arguments before evaluating when constant

Repository: phoenix
Updated Branches:
  refs/heads/master 3fb3bb4d2 -> e25d7d098


PHOENIX-1876 Check for null function arguments before evaluating when constant


Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/035e3157
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/035e3157
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/035e3157

Branch: refs/heads/master
Commit: 035e315794427e0914be6d0a84fb6bac5331d6a8
Parents: 3fb3bb4
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Apr 16 10:01:25 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Apr 16 10:01:25 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/phoenix/end2end/DecodeFunctionIT.java   | 9 +++------
 .../java/org/apache/phoenix/end2end/EncodeFunctionIT.java   | 8 ++------
 .../java/org/apache/phoenix/compile/ExpressionCompiler.java | 6 +++---
 3 files changed, 8 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/035e3157/phoenix-core/src/it/java/org/apache/phoenix/end2end/DecodeFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DecodeFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DecodeFunctionIT.java
index 68e0add..93205a7 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DecodeFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DecodeFunctionIT.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.end2end;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -111,12 +112,8 @@ public class DecodeFunctionIT extends BaseHBaseManagedTimeIT {
 
 		conn.createStatement().execute(ddl);
 
-		try {
-			conn.createStatement().executeQuery("SELECT * FROM test_table WHERE some_column = DECODE('8', NULL)");
-            fail();
-        } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.ILLEGAL_DATA.getErrorCode(), e.getErrorCode());
-        }
+		ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM test_table WHERE some_column = DECODE('8', NULL)");
+		assertFalse(rs.next());
 	}
 
 	@Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/035e3157/phoenix-core/src/it/java/org/apache/phoenix/end2end/EncodeFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/EncodeFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/EncodeFunctionIT.java
index 489ebfd..ceafc5b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/EncodeFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/EncodeFunctionIT.java
@@ -116,12 +116,8 @@ public class EncodeFunctionIT extends BaseHBaseManagedTimeIT {
         String ddl = "CREATE TABLE TEST_TABLE ( pk VARCHAR(10) NOT NULL CONSTRAINT PK PRIMARY KEY (pk))";
         conn.createStatement().execute(ddl);
 
-        try {
-            conn.createStatement().executeQuery("SELECT * FROM TEST_TABLE WHERE pk = ENCODE(1, NULL)");
-            fail();
-        } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.ILLEGAL_DATA.getErrorCode(), e.getErrorCode());
-        }
+        ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM TEST_TABLE WHERE pk = ENCODE(1, NULL)");
+        assertFalse(rs.next());
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/035e3157/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
index ce95850..ab6b851 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
@@ -316,9 +316,6 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
         children = node.validate(children, context);
         Expression expression = node.create(children, context);
         ImmutableBytesWritable ptr = context.getTempPtr();
-        if (ExpressionUtil.isConstant(expression)) {
-            return ExpressionUtil.getConstantExpression(expression, ptr);
-        }
         BuiltInFunctionInfo info = node.getInfo();
         for (int i = 0; i < info.getRequiredArgCount(); i++) { 
             // Optimization to catch cases where a required argument is null resulting in the function
@@ -331,6 +328,9 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
                 }
             }
         }
+        if (ExpressionUtil.isConstant(expression)) {
+            return ExpressionUtil.getConstantExpression(expression, ptr);
+        }
         expression = addExpression(expression);
         expression = wrapGroupByExpression(expression);
         if (aggregateFunction == node) {


[2/2] phoenix git commit: PHOENIX-1870 Fix NPE occurring during regex processing when joni library not used

Posted by ja...@apache.org.
PHOENIX-1870 Fix NPE occurring during regex processing when joni library not used


Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/e25d7d09
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/e25d7d09
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/e25d7d09

Branch: refs/heads/master
Commit: e25d7d098c7d537fe8f3ee36838664c26f52a5ac
Parents: 035e315
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Apr 16 10:11:13 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Apr 16 10:11:13 2015 -0700

----------------------------------------------------------------------
 .../function/RegexpSubstrFunction.java          | 97 ++++++++++++--------
 .../util/regex/AbstractBasePattern.java         |  3 +-
 .../expression/util/regex/JONIPattern.java      | 18 ++--
 .../expression/util/regex/JavaPattern.java      | 31 ++++---
 .../util/regex/PatternPerformanceTest.java      |  7 +-
 5 files changed, 92 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e25d7d09/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
index 430b444..ea80b11 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
@@ -22,8 +22,8 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Determinism;
 import org.apache.phoenix.expression.Expression;
-import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
 import org.apache.phoenix.parse.FunctionParseNode.Argument;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
@@ -31,6 +31,7 @@ import org.apache.phoenix.parse.RegexpSubstrParseNode;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PVarchar;
 
@@ -56,11 +57,11 @@ public abstract class RegexpSubstrFunction extends PrefixFunction {
     public static final String NAME = "REGEXP_SUBSTR";
 
     private AbstractBasePattern pattern;
-    private boolean isOffsetConstant;
+    private Integer offset;
     private Integer maxLength;
 
     private static final PDataType TYPE = PVarchar.INSTANCE;
-
+    
     public RegexpSubstrFunction() { }
 
     public RegexpSubstrFunction(List<Expression> children) {
@@ -71,25 +72,30 @@ public abstract class RegexpSubstrFunction extends PrefixFunction {
     protected abstract AbstractBasePattern compilePatternSpec(String value);
 
     private void init() {
-        Object patternString = ((LiteralExpression)children.get(1)).getValue();
-        if (patternString != null) {
-            pattern = compilePatternSpec((String) patternString);
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        Expression patternExpr = getPatternExpression();
+        if (patternExpr.isStateless() && patternExpr.getDeterminism() == Determinism.ALWAYS && patternExpr.evaluate(null, ptr)) {
+            String patternStr = (String) patternExpr.getDataType().toObject(ptr, patternExpr.getSortOrder());
+            if (patternStr != null) {
+                pattern = compilePatternSpec(patternStr);
+            }
         }
         // If the source string has a fixed width, then the max length would be the length 
         // of the source string minus the offset, or the absolute value of the offset if 
         // it's negative. Offset number is a required argument. However, if the source string
         // is not fixed width, the maxLength would be null.
-        isOffsetConstant = getOffsetExpression() instanceof LiteralExpression;
-        Number offsetNumber = (Number)((LiteralExpression)getOffsetExpression()).getValue();
-        if (offsetNumber != null) {
-            int offset = offsetNumber.intValue();
-            PDataType type = getSourceStrExpression().getDataType();
-            if (type.isFixedWidth()) {
-                if (offset >= 0) {
-                    Integer maxLength = getSourceStrExpression().getMaxLength();
-                    this.maxLength = maxLength - offset - (offset == 0 ? 0 : 1);
-                } else {
-                    this.maxLength = -offset;
+        Expression offsetExpr = getOffsetExpression();
+        if (offsetExpr.isStateless() && offsetExpr.getDeterminism() == Determinism.ALWAYS && offsetExpr.evaluate(null, ptr)) {
+            offset = (Integer)PInteger.INSTANCE.toObject(ptr, offsetExpr.getDataType(), offsetExpr.getSortOrder());
+            if (offset != null) {
+                PDataType type = getSourceStrExpression().getDataType();
+                if (type.isFixedWidth()) {
+                    if (offset >= 0) {
+                        Integer maxLength = getSourceStrExpression().getMaxLength();
+                        this.maxLength = maxLength - offset - (offset == 0 ? 0 : 1);
+                    } else {
+                        this.maxLength = -offset;
+                    }
                 }
             }
         }
@@ -97,25 +103,45 @@ public abstract class RegexpSubstrFunction extends PrefixFunction {
 
     @Override
     public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        AbstractBasePattern pattern = this.pattern;
         if (pattern == null) {
-            return false;
+            Expression patternExpr = getPatternExpression();
+            if (!patternExpr.evaluate(tuple, ptr)) {
+                return false;
+            }
+            if (ptr.getLength() == 0) {
+                return true;
+            }
+            pattern = compilePatternSpec((String) patternExpr.getDataType().toObject(ptr, patternExpr.getSortOrder()));
         }
-        ImmutableBytesWritable srcPtr = new ImmutableBytesWritable();
-        if (!getSourceStrExpression().evaluate(tuple, srcPtr)) {
-            return false;
+        int offset;
+        if (this.offset == null) {
+            Expression offsetExpression = getOffsetExpression();
+            if (!offsetExpression.evaluate(tuple, ptr)) {
+                return false;
+            }
+            if (ptr.getLength() == 0) {
+                return true;
+            }
+            offset = offsetExpression.getDataType().getCodec().decodeInt(ptr, offsetExpression.getSortOrder());
+        } else {
+            offset = this.offset;
         }
-        TYPE.coerceBytes(srcPtr, TYPE, getSourceStrExpression().getSortOrder(), SortOrder.ASC);
-
-        Expression offsetExpression = getOffsetExpression();
-        if (!offsetExpression.evaluate(tuple, ptr)) {
+        Expression strExpression = getSourceStrExpression();
+        if (!strExpression.evaluate(tuple, ptr)) {
             return false;
         }
-        int offset = offsetExpression.getDataType().getCodec().decodeInt(ptr, offsetExpression.getSortOrder());
+        if (ptr.get().length == 0) {
+            return true;
+        }
+
+        TYPE.coerceBytes(ptr, strExpression.getDataType(), strExpression.getSortOrder(), SortOrder.ASC);
 
         // Account for 1 versus 0-based offset
         offset = offset - (offset <= 0 ? 0 : 1);
 
-        return pattern.substr(srcPtr, offset, ptr);
+        pattern.substr(ptr, offset);
+        return true;
     }
 
     @Override
@@ -125,14 +151,9 @@ public abstract class RegexpSubstrFunction extends PrefixFunction {
 
     @Override
     public OrderPreserving preservesOrder() {
-        if (isOffsetConstant) {
-            LiteralExpression literal = (LiteralExpression) getOffsetExpression();
-            Number offsetNumber = (Number) literal.getValue();
-            if (offsetNumber != null) { 
-                int offset = offsetNumber.intValue();
-                if (offset == 0 || offset == 1) {
-                    return OrderPreserving.YES_IF_LAST;
-                }
+        if (offset != null) {
+            if (offset == 0 || offset == 1) {
+                return OrderPreserving.YES_IF_LAST;
             }
         }
         return OrderPreserving.NO;
@@ -153,6 +174,10 @@ public abstract class RegexpSubstrFunction extends PrefixFunction {
         return children.get(2);
     }
 
+    private Expression getPatternExpression() {
+        return children.get(1);
+    }
+
     private Expression getSourceStrExpression() {
         return children.get(0);
     }
@@ -161,7 +186,7 @@ public abstract class RegexpSubstrFunction extends PrefixFunction {
     public PDataType getDataType() {
         // ALways VARCHAR since we do not know in advanced how long the 
         // matched string will be.
-        return PVarchar.INSTANCE;
+        return TYPE;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e25d7d09/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/AbstractBasePattern.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/AbstractBasePattern.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/AbstractBasePattern.java
index 27b47a0..5287fd7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/AbstractBasePattern.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/AbstractBasePattern.java
@@ -26,8 +26,7 @@ public abstract class AbstractBasePattern {
     public abstract void replaceAll(ImmutableBytesWritable srcPtr,
             ImmutableBytesWritable replacePtr, ImmutableBytesWritable outPtr);
 
-    public abstract boolean substr(ImmutableBytesWritable srcPtr, int offsetInStr,
-            ImmutableBytesWritable outPtr);
+    public abstract void substr(ImmutableBytesWritable srcPtr, int offsetInStr);
 
     public abstract String pattern();
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e25d7d09/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JONIPattern.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JONIPattern.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JONIPattern.java
index 5c0b1bc..b17e8a7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JONIPattern.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JONIPattern.java
@@ -130,15 +130,15 @@ public class JONIPattern extends AbstractBasePattern implements AbstractBaseSpli
     }
 
     @Override
-    public boolean substr(ImmutableBytesWritable srcPtr, int offsetInStr,
-            ImmutableBytesWritable outPtr) {
-        Preconditions.checkNotNull(srcPtr);
-        Preconditions.checkNotNull(outPtr);
-        int offsetInBytes = StringUtil.calculateUTF8Offset(srcPtr.get(), srcPtr.getOffset(),
-            srcPtr.getLength(), SortOrder.ASC, offsetInStr);
-        if (offsetInBytes < 0) return false;
-        substr(srcPtr.get(), offsetInBytes, srcPtr.getOffset() + srcPtr.getLength(), outPtr);
-        return true;
+    public void substr(ImmutableBytesWritable ptr, int offsetInStr) {
+        Preconditions.checkNotNull(ptr);
+        int offsetInBytes = StringUtil.calculateUTF8Offset(ptr.get(), ptr.getOffset(),
+                ptr.getLength(), SortOrder.ASC, offsetInStr);
+        if (offsetInBytes < 0) {
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+        } else {
+            substr(ptr.get(), offsetInBytes, ptr.getOffset() + ptr.getLength(), ptr);
+        }
     }
 
     private boolean substr(byte[] srcBytes, int offset, int range, ImmutableBytesWritable outPtr) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e25d7d09/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JavaPattern.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JavaPattern.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JavaPattern.java
index be1188c..f4bd239 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JavaPattern.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JavaPattern.java
@@ -73,21 +73,24 @@ public class JavaPattern extends AbstractBasePattern {
     }
 
     @Override
-    public boolean substr(ImmutableBytesWritable srcPtr, int offsetInStr,
-            ImmutableBytesWritable outPtr) {
-        Preconditions.checkNotNull(srcPtr);
-        Preconditions.checkNotNull(outPtr);
-        String sourceStr = (String) PVarchar.INSTANCE.toObject(srcPtr);
-        if (srcPtr.get().length == 0 && sourceStr == null) sourceStr = "";
-        if (offsetInStr < 0) offsetInStr += sourceStr.length();
-        if (offsetInStr < 0 || offsetInStr >= sourceStr.length()) return false;
-        Matcher matcher = pattern.matcher(sourceStr);
-        boolean ret = matcher.find(offsetInStr);
-        if (ret) {
-            outPtr.set(PVarchar.INSTANCE.toBytes(matcher.group()));
+    public void substr(ImmutableBytesWritable ptr, int offsetInStr) {
+        Preconditions.checkNotNull(ptr);
+        String sourceStr = (String) PVarchar.INSTANCE.toObject(ptr);
+        if (sourceStr == null) {
+            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
         } else {
-            outPtr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            if (offsetInStr < 0) offsetInStr += sourceStr.length();
+            if (offsetInStr < 0 || offsetInStr >= sourceStr.length()) {
+                ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            } else {
+                Matcher matcher = pattern.matcher(sourceStr);
+                boolean ret = matcher.find(offsetInStr);
+                if (ret) {
+                    ptr.set(PVarchar.INSTANCE.toBytes(matcher.group()));
+                } else {
+                    ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+                }
+            }
         }
-        return true;
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e25d7d09/phoenix-core/src/test/java/org/apache/phoenix/expression/util/regex/PatternPerformanceTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/util/regex/PatternPerformanceTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/util/regex/PatternPerformanceTest.java
index 908c662..4275687 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/expression/util/regex/PatternPerformanceTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/util/regex/PatternPerformanceTest.java
@@ -101,10 +101,11 @@ public class PatternPerformanceTest {
     private void testSubstr(AbstractBasePattern pattern, String name) {
         timer.reset();
         for (int i = 0; i < maxTimes; ++i) {
-            boolean ret = pattern.substr(dataPtr[i % 3], 0, resultPtr);
+            ImmutableBytesWritable ptr = dataPtr[i % 3];
+            resultPtr.set(ptr.get(),ptr.getOffset(),ptr.getLength());
+            pattern.substr(resultPtr, 0);
             if (ENABLE_ASSERT) {
-                assertTrue(ret
-                        && (i % 3 != 2 || ":THU".equals(PVarchar.INSTANCE.toObject(resultPtr))));
+                assertTrue((i % 3 != 2 || ":THU".equals(PVarchar.INSTANCE.toObject(resultPtr))));
             }
         }
         timer.printTime(name);