You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ra...@apache.org on 2016/10/27 07:56:40 UTC

phoenix git commit: PHOENIX-3242 Support UDF in Phoenix-Calcite Integration-addendum(Rajeshbabu)

Repository: phoenix
Updated Branches:
  refs/heads/calcite 6457af186 -> 0b156dc64


PHOENIX-3242 Support UDF in Phoenix-Calcite Integration-addendum(Rajeshbabu)


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

Branch: refs/heads/calcite
Commit: 0b156dc64612e2df2914e423f4ed872cce40ab8c
Parents: 6457af1
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Thu Oct 27 13:36:19 2016 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Thu Oct 27 13:36:19 2016 +0530

----------------------------------------------------------------------
 .../phoenix/end2end/UserDefinedFunctionsIT.java |  5 +-
 .../apache/phoenix/calcite/CalciteUtils.java    | 27 +++++++-
 .../apache/phoenix/parse/FunctionParseNode.java | 73 +++++++++++---------
 3 files changed, 69 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b156dc6/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
index f8d36f6..dfd4de9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
@@ -619,7 +619,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
     public void testDropFunction() throws Exception {
         Connection conn = driver.connect(calciteUrl, UDF_PROPS);
         Statement stmt = conn.createStatement();
-        String query = "select count(*) from "+ SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\"";
+        String query = "select count(*) from "+"\""+ SYSTEM_CATALOG_SCHEMA + "\".\"" + SYSTEM_FUNCTION_TABLE + "\"";
         ResultSet rs = stmt.executeQuery(query);
         rs.next();
         int numRowsBefore = rs.getInt(1);
@@ -665,7 +665,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
         Connection conn = DriverManager.getConnection(calciteUrl, props);
         Statement stmt = conn.createStatement();
-        String query = "select count(*) from "+ SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\"";
+        String query = "select count(*) from \""+ SYSTEM_CATALOG_SCHEMA + "\".\"" + SYSTEM_FUNCTION_TABLE + "\"";
         ResultSet rs = stmt.executeQuery(query);
         rs.next();
         int numRowsBefore = rs.getInt(1);
@@ -673,6 +673,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
                 + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar2.jar"+"'");
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         conn = DriverManager.getConnection(calciteUrl, props);
+        stmt = conn.createStatement();
         rs = stmt.executeQuery(query);
         rs.next();
         int numRowsAfter= rs.getInt(1);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b156dc6/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
index 7e6bd20..b312954 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteUtils.java
@@ -2,6 +2,7 @@ package org.apache.phoenix.calcite;
 
 import java.sql.SQLException;
 import java.sql.Timestamp;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.GregorianCalendar;
 import java.util.List;
@@ -107,6 +108,8 @@ import org.apache.phoenix.expression.function.SumAggregateFunction;
 import org.apache.phoenix.expression.function.TrimFunction;
 import org.apache.phoenix.expression.function.UDFExpression;
 import org.apache.phoenix.expression.function.UpperFunction;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunctionInfo;
 import org.apache.phoenix.parse.JoinTableNode.JoinType;
 import org.apache.phoenix.parse.SequenceValueParseNode;
 import org.apache.phoenix.schema.PTableType;
@@ -734,6 +737,13 @@ public class CalciteUtils {
                 }
             }
         });
+        EXPRESSION_MAP.put(SqlKind.DEFAULT, new ExpressionFactory() {
+            @SuppressWarnings("rawtypes")
+            @Override
+            public Expression newExpression(RexNode node, PhoenixRelImplementor implementor) {
+                return null;
+            }
+        });
         EXPRESSION_MAP.put(SqlKind.OTHER_FUNCTION, new ExpressionFactory() {
             @Override
             public Expression newExpression(RexNode node,
@@ -747,6 +757,19 @@ public class CalciteUtils {
                         Function func = udf.getFunction();
                         if (func instanceof PhoenixScalarFunction) {
                             PhoenixScalarFunction scalarFunc = (PhoenixScalarFunction) func;
+                            BuiltInFunctionInfo info = new BuiltInFunctionInfo(scalarFunc.getFunctionInfo());
+                            if (info.getArgs().length > children.size()) {
+                                List<Expression> moreChildren = new ArrayList<Expression>(children);
+                                for (int i = children.size(); i < info.getArgs().length; i++) {
+                                    if(info.getArgs()[i].getDefaultValue() != null) {
+                                        moreChildren.add(info.getArgs()[i].getDefaultValue());
+                                    }
+                                }
+                                children = moreChildren;
+                            }
+                            for(int i = 0; i < children.size(); i++) {
+                                FunctionParseNode.validateFunctionArguement(info, i, children.get(i));
+                            }
                             return new UDFExpression(children, scalarFunc.getFunctionInfo());
                         }
                     } else if (op == SqlStdOperatorTable.SQRT) {
@@ -966,7 +989,9 @@ public class CalciteUtils {
         List<Expression> children = Lists.newArrayListWithExpectedSize(call.getOperands().size());
         for (RexNode op : call.getOperands()) {
             Expression child = getFactory(op).newExpression(op, implementor);
-            children.add(child);
+            if(child != null) {
+                children.add(child);
+            }
         }
         return children;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b156dc6/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
index 0dd021b..952d0d3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
@@ -186,44 +186,51 @@ public class FunctionParseNode extends CompoundParseNode {
                     }
                 }
             } else {
-                if (allowedTypes.length > 0) {
-                    boolean isCoercible = false;
-                    for (Class<? extends PDataType> type : allowedTypes) {
-                        if (child.getDataType().isCoercibleTo(
-                            PDataTypeFactory.getInstance().instanceFromClass(type))) {
-                            isCoercible = true;
-                            break;
-                        }
-                    }
-                    if (!isCoercible) {
-                        throw new ArgumentTypeMismatchException(args[i].getAllowedTypes(),
-                            child.getDataType(), info.getName() + " argument " + (i + 1));
-                    }
-                    if (child instanceof LiteralExpression) {
-                        LiteralExpression valueExp = (LiteralExpression) child;
-                        LiteralExpression minValue = args[i].getMinValue();
-                        LiteralExpression maxValue = args[i].getMaxValue();
-                        if (minValue != null && minValue.getDataType().compareTo(minValue.getValue(), valueExp.getValue(), valueExp.getDataType()) > 0) {
-                            throw new ValueRangeExcpetion(minValue, maxValue == null ? "" : maxValue, valueExp.getValue(), info.getName() + " argument " + (i + 1));
-                        }
-                        if (maxValue != null && maxValue.getDataType().compareTo(maxValue.getValue(), valueExp.getValue(), valueExp.getDataType()) < 0) {
-                            throw new ValueRangeExcpetion(minValue == null ? "" : minValue, maxValue, valueExp.getValue(), info.getName() + " argument " + (i + 1));
-                        }
-                    }
+                validateFunctionArguement(info, i, child);
+            }
+        }
+        return children;
+    }
+
+    public static void validateFunctionArguement(BuiltInFunctionInfo info,
+            int childIndex, Expression child)
+            throws ArgumentTypeMismatchException, ValueRangeExcpetion {
+        BuiltInFunctionArgInfo arg = info.getArgs()[childIndex];
+        if (arg.getAllowedTypes().length > 0) {
+            boolean isCoercible = false;
+            for (Class<? extends PDataType> type :arg.getAllowedTypes()) {
+                if (child.getDataType().isCoercibleTo(
+                    PDataTypeFactory.getInstance().instanceFromClass(type))) {
+                    isCoercible = true;
+                    break;
                 }
-                if (args[i].isConstant() && ! (child instanceof LiteralExpression) ) {
-                    throw new ArgumentTypeMismatchException("constant", child.toString(), info.getName() + " argument " + (i + 1));
+            }
+            if (!isCoercible) {
+                throw new ArgumentTypeMismatchException(arg.getAllowedTypes(),
+                    child.getDataType(), info.getName() + " argument " + (childIndex + 1));
+            }
+            if (child instanceof LiteralExpression) {
+                LiteralExpression valueExp = (LiteralExpression) child;
+                LiteralExpression minValue = arg.getMinValue();
+                LiteralExpression maxValue = arg.getMaxValue();
+                if (minValue != null && minValue.getDataType().compareTo(minValue.getValue(), valueExp.getValue(), valueExp.getDataType()) > 0) {
+                    throw new ValueRangeExcpetion(minValue, maxValue == null ? "" : maxValue, valueExp.getValue(), info.getName() + " argument " + (childIndex + 1));
                 }
-                if (!args[i].getAllowedValues().isEmpty()) {
-                    Object value = ((LiteralExpression)child).getValue();
-                    if (!args[i].getAllowedValues().contains(value.toString().toUpperCase())) {
-                        throw new ArgumentTypeMismatchException(Arrays.toString(args[i].getAllowedValues().toArray(new String[0])),
-                                value.toString(), info.getName() + " argument " + (i + 1));
-                    }
+                if (maxValue != null && maxValue.getDataType().compareTo(maxValue.getValue(), valueExp.getValue(), valueExp.getDataType()) < 0) {
+                    throw new ValueRangeExcpetion(minValue == null ? "" : minValue, maxValue, valueExp.getValue(), info.getName() + " argument " + (childIndex + 1));
                 }
             }
         }
-        return children;
+        if (arg.isConstant() && ! (child instanceof LiteralExpression) ) {
+            throw new ArgumentTypeMismatchException("constant", child.toString(), info.getName() + " argument " + (childIndex + 1));
+        }
+        if (!arg.getAllowedValues().isEmpty()) {
+            Object value = ((LiteralExpression)child).getValue();
+            if (!arg.getAllowedValues().contains(value.toString().toUpperCase())) {
+                throw new ArgumentTypeMismatchException(Arrays.toString(arg.getAllowedValues().toArray(new String[0])),
+                        value.toString(), info.getName() + " argument " + (childIndex + 1));
+            }
+        }
     }
 
     /**