You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by al...@apache.org on 2019/05/24 22:54:48 UTC

[asterixdb] branch master updated: [ASTERIXDB-2458][COMP] Fix InjectTypeCastForFunctionArgumentsRule

This is an automated email from the ASF dual-hosted git repository.

alsuliman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 1a565f8  [ASTERIXDB-2458][COMP] Fix InjectTypeCastForFunctionArgumentsRule
1a565f8 is described below

commit 1a565f89959ba281dc0dfd8c9664fc5a373f09ad
Author: Ali Alsuliman <al...@gmail.com>
AuthorDate: Fri May 24 11:56:08 2019 -0700

    [ASTERIXDB-2458][COMP] Fix InjectTypeCastForFunctionArgumentsRule
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    InjectTypeCastForFunctionArgumentsRule is for functions that
    can potentially return any of their arguments. switch and
    if_null(expr1, expr2, ...) are examples. All the arguments
    need to be casted (opened) to the type that the function
    will return which is the generalized type of all arguments.
    Some functions like if_null can determine the exact expression
    they will return, e.g. if_null(1, {"id": 3}) in which case
    the return type is always integer. The rule tries to cast
    th 2nd argument, the record, to integer and fails. In such
    cases, these functions do not need to cast their arguments.
    If the function determines its output type to be ANY, then
    all arguments need to be casted (opened). If the function
    determines its output to be a dervied type, then casting is
    also needed since that output type should be the generalized
    type of all arguments.
    
    Change-Id: I2fee234d883b59319e4ec4df58d61ecd498373fd
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/3406
    Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
---
 .../InjectTypeCastForFunctionArgumentsRule.java    | 47 ++++++++++++----------
 .../cast-ASTERIXDB-2458.1.query.sqlpp              | 26 ++++++++++++
 .../cast-ASTERIXDB-2458/cast-ASTERIXDB-2458.1.adm  |  6 +++
 .../test/resources/runtimets/testsuite_sqlpp.xml   |  5 +++
 .../dataflow/data/common/TypeResolverUtil.java     |  2 +-
 5 files changed, 64 insertions(+), 22 deletions(-)

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForFunctionArgumentsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForFunctionArgumentsRule.java
index 3678826..77d63f1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForFunctionArgumentsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForFunctionArgumentsRule.java
@@ -24,12 +24,15 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.function.IntBinaryOperator;
 import java.util.function.IntPredicate;
 
 import org.apache.asterix.dataflow.data.common.TypeResolverUtil;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
@@ -103,39 +106,27 @@ public class InjectTypeCastForFunctionArgumentsRule implements IAlgebraicRewrite
         }
         FunctionIdentifier funcId = func.getFunctionIdentifier();
         if (funcId.equals(BuiltinFunctions.SWITCH_CASE)) {
-            rewritten |= rewriteSwitchCase(op, func, context);
+            rewritten |= rewriteFunction(op, func, null, context, 2,
+                    InjectTypeCastForFunctionArgumentsRule::switchIncrement);
         } else if (FUN_TO_ARG_CHECKER.containsKey(funcId)) {
-            rewritten |= rewriteFunction(op, func, FUN_TO_ARG_CHECKER.get(funcId), context);
-        }
-        return rewritten;
-    }
-
-    // Injects casts that cast types for different "THEN" and "ELSE" branches.
-    private boolean rewriteSwitchCase(ILogicalOperator op, AbstractFunctionCallExpression func,
-            IOptimizationContext context) throws AlgebricksException {
-        IVariableTypeEnvironment env = op.computeInputTypeEnvironment(context);
-        IAType producedType = (IAType) env.getType(func);
-        List<Mutable<ILogicalExpression>> argRefs = func.getArguments();
-        int argSize = argRefs.size();
-        boolean rewritten = false;
-        for (int argIndex = 2; argIndex < argSize; argIndex += (argIndex + 2 == argSize) ? 1 : 2) {
-            Mutable<ILogicalExpression> argRef = argRefs.get(argIndex);
-            if (rewriteFunctionArgument(argRef, producedType, env)) {
-                rewritten = true;
-            }
+            rewritten |= rewriteFunction(op, func, FUN_TO_ARG_CHECKER.get(funcId), context, 0,
+                    InjectTypeCastForFunctionArgumentsRule::increment);
         }
         return rewritten;
     }
 
     // Injects casts that cast types for all function parameters
     private boolean rewriteFunction(ILogicalOperator op, AbstractFunctionCallExpression func, IntPredicate argChecker,
-            IOptimizationContext context) throws AlgebricksException {
+            IOptimizationContext context, int argStartIdx, IntBinaryOperator increment) throws AlgebricksException {
         IVariableTypeEnvironment env = op.computeInputTypeEnvironment(context);
         IAType producedType = (IAType) env.getType(func);
+        if (!argumentsNeedCasting(producedType)) {
+            return false;
+        }
         List<Mutable<ILogicalExpression>> argRefs = func.getArguments();
         int argSize = argRefs.size();
         boolean rewritten = false;
-        for (int argIndex = 0; argIndex < argSize; argIndex++) {
+        for (int argIndex = argStartIdx; argIndex < argSize; argIndex += increment.applyAsInt(argIndex, argSize)) {
             if (argChecker == null || argChecker.test(argIndex)) {
                 rewritten |= rewriteFunctionArgument(argRefs.get(argIndex), producedType, env);
             }
@@ -159,4 +150,18 @@ public class InjectTypeCastForFunctionArgumentsRule implements IAlgebraicRewrite
         }
         return false;
     }
+
+    private static boolean argumentsNeedCasting(IAType functionProducedType) {
+        ATypeTag functionProducedTag = TypeComputeUtils.getActualType(functionProducedType).getTypeTag();
+        return functionProducedTag == ATypeTag.ANY || functionProducedTag.isDerivedType();
+    }
+
+    private static int switchIncrement(int currentArgIndex, int numArguments) {
+        return currentArgIndex + 2 == numArguments ? 1 : 2;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    private static int increment(int currentArgIndex, int numArguments) {
+        return 1;
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/cast-ASTERIXDB-2458/cast-ASTERIXDB-2458.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/cast-ASTERIXDB-2458/cast-ASTERIXDB-2458.1.query.sqlpp
new file mode 100644
index 0000000..cb7f81e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/cast-ASTERIXDB-2458/cast-ASTERIXDB-2458.1.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+FROM [
+if_missing(1, {"id": 4}),
+if_missing({"id": 4}, 1),
+if_missing_or_null(1, {"id": 4}),
+if_missing_or_null({"id": 4}, 1),
+if_null(1, {"id": 4}),
+if_null({"id": 4}, 1)] as v select v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/cast-ASTERIXDB-2458/cast-ASTERIXDB-2458.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/cast-ASTERIXDB-2458/cast-ASTERIXDB-2458.1.adm
new file mode 100644
index 0000000..07e8234
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/cast-ASTERIXDB-2458/cast-ASTERIXDB-2458.1.adm
@@ -0,0 +1,6 @@
+{ "v": 1 }
+{ "v": { "id": 4 } }
+{ "v": 1 }
+{ "v": { "id": 4 } }
+{ "v": 1 }
+{ "v": { "id": 4 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 1d21dfa..fae2795 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -6032,6 +6032,11 @@
         <output-dir compare="Text">metadata_only_01</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="cast-ASTERIXDB-2458">
+        <output-dir compare="Text">cast-ASTERIXDB-2458</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="index">
     <test-group name="index/validations">
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java
index b7bd8ca..3cc9e3d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java
@@ -89,7 +89,7 @@ public class TypeResolverUtil {
         }
         // Casts are only needed when the original return type is a complex type.
         // (In the runtime, there is already a type tag for scalar types.)
-        if (tag != ATypeTag.OBJECT && tag != ATypeTag.MULTISET && tag != ATypeTag.ARRAY) {
+        if (!tag.isDerivedType()) {
             return false;
         }
         return !TypeComputeUtils.getActualType(reqType).equals(TypeComputeUtils.getActualType(inputType));