You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mb...@apache.org on 2019/10/29 18:20:39 UTC

[asterixdb] 03/06: [NO ISSUE][COMP] Fix incorrect error with if_*() functions

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

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

commit b079ed08c80f31ab37a9908247f19034fe27b2a4
Author: Dmitry Lychagin <dm...@couchbase.com>
AuthorDate: Thu Oct 24 13:12:24 2019 -0700

    [NO ISSUE][COMP] Fix incorrect error with if_*() functions
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    - Fixed incorrect 'Type incompatibility' error raised by
      the compiler when arguments of if_null(), if_missing(),
      and if_missing_or_null() have different complex types
      (array and record)
    
    Change-Id: Idb285a3314604cd19a76d011a4739476fb7c7d7c
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/3903
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Ali Alsuliman <al...@gmail.com>
---
 .../InjectTypeCastForFunctionArgumentsRule.java    |  3 +-
 .../null-missing/ifmissing/ifmissing.1.query.sqlpp |  3 +-
 .../ifmissingornull/ifmissingornull.1.query.sqlpp  |  3 +-
 .../null-missing/ifnull/ifnull.1.query.sqlpp       |  3 +-
 .../results/null-missing/ifmissing/ifmissing.1.adm |  2 +-
 .../ifmissingornull/ifmissingornull.1.adm          |  2 +-
 .../results/null-missing/ifnull/ifnull.1.adm       |  2 +-
 .../asterix/om/pointables/cast/ACastVisitor.java   | 57 +++++++++++++++++-----
 .../om/typecomputer/base/TypeCastUtils.java        | 18 +++++--
 .../evaluators/functions/CastTypeDescriptor.java   |  3 +-
 .../evaluators/functions/CastTypeEvaluator.java    | 16 +++++-
 .../functions/CastTypeLaxDescriptor.java           |  3 +-
 .../evaluators/functions/CastTypeLaxEvaluator.java |  7 +--
 13 files changed, 91 insertions(+), 31 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 77d63f1..9531e94 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
@@ -144,7 +144,8 @@ public class InjectTypeCastForFunctionArgumentsRule implements IAlgebraicRewrite
                     new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE),
                             new ArrayList<>(Collections.singletonList(new MutableObject<>(argExpr))));
             castFunc.setSourceLocation(argExpr.getSourceLocation());
-            TypeCastUtils.setRequiredAndInputTypes(castFunc, funcOutputType, type);
+            IAType funcOutputPrimeType = TypeComputeUtils.getActualType(funcOutputType);
+            TypeCastUtils.setRequiredAndInputTypes(castFunc, funcOutputPrimeType, type, false);
             argRef.setValue(castFunc);
             return true;
         }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissing/ifmissing.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissing/ifmissing.1.query.sqlpp
index 0abb997..f0c0001 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissing/ifmissing.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissing/ifmissing.1.query.sqlpp
@@ -36,5 +36,6 @@
                 { "c": [ 2 ] }
             )
             select v as b
-        )
+        ),
+   "k": if_missing(missing, ["a"], "abc", {"id":3})
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissingornull/ifmissingornull.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissingornull/ifmissingornull.1.query.sqlpp
index 22a8acd..735cfa9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissingornull/ifmissingornull.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissingornull/ifmissingornull.1.query.sqlpp
@@ -39,5 +39,6 @@
                 { "c": [ 2 ] }
             )
             select v as b
-        )
+        ),
+   "k": if_missing_or_null(null, missing, ["a"], "abc", {"id":3})
 };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifnull/ifnull.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifnull/ifnull.1.query.sqlpp
index 0121cd8..6ae39e5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifnull/ifnull.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifnull/ifnull.1.query.sqlpp
@@ -37,5 +37,6 @@
                 { "c": [ 2 ] }
             )
             select v as b
-        )
+        ),
+   "k": if_null(null,["a"],"abc", {"id":3})
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissing/ifmissing.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissing/ifmissing.1.adm
index 14620a1..f416057 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissing/ifmissing.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissing/ifmissing.1.adm
@@ -1 +1 @@
-{ "a": true, "b": true, "c": true, "d": true, "e": true, "f": true, "g": true, "i": true, "j": [ { "b": { "c": [ 2 ] } } ] }
\ No newline at end of file
+{ "a": true, "b": true, "c": true, "d": true, "e": true, "f": true, "g": true, "i": true, "j": [ { "b": { "c": [ 2 ] } } ], "k": [ "a" ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissingornull/ifmissingornull.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissingornull/ifmissingornull.1.adm
index eff2651..ff4b01f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissingornull/ifmissingornull.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissingornull/ifmissingornull.1.adm
@@ -1 +1 @@
-{ "a": true, "b": true, "c": true, "d": true, "e": true, "f": true, "g": true, "h": true, "i": true, "j": [ { "b": { "c": [ 2 ] } } ] }
\ No newline at end of file
+{ "a": true, "b": true, "c": true, "d": true, "e": true, "f": true, "g": true, "h": true, "i": true, "j": [ { "b": { "c": [ 2 ] } } ], "k": [ "a" ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifnull/ifnull.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifnull/ifnull.1.adm
index eff2651..ff4b01f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifnull/ifnull.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifnull/ifnull.1.adm
@@ -1 +1 @@
-{ "a": true, "b": true, "c": true, "d": true, "e": true, "f": true, "g": true, "h": true, "i": true, "j": [ { "b": { "c": [ 2 ] } } ] }
\ No newline at end of file
+{ "a": true, "b": true, "c": true, "d": true, "e": true, "f": true, "g": true, "h": true, "i": true, "j": [ { "b": { "c": [ 2 ] } } ], "k": [ "a" ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
index 04ee28b..993edee 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
@@ -23,6 +23,8 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.pointables.AFlatValuePointable;
 import org.apache.asterix.om.pointables.AListVisitablePointable;
 import org.apache.asterix.om.pointables.ARecordVisitablePointable;
@@ -37,6 +39,7 @@ import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 
 /**
@@ -57,13 +60,19 @@ public class ACastVisitor implements IVisitablePointableVisitor<Void, Triple<IVi
     private final ArrayBackedValueStorage castBuffer = new ArrayBackedValueStorage();
 
     private final boolean strictDemote;
+    private final SourceLocation sourceLoc;
 
     public ACastVisitor() {
-        this(true);
+        this(true, null);
     }
 
-    public ACastVisitor(boolean strictDemote) {
+    public ACastVisitor(SourceLocation sourceLoc) {
+        this(true, sourceLoc);
+    }
+
+    public ACastVisitor(boolean strictDemote, SourceLocation sourceLoc) {
         this.strictDemote = strictDemote;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -74,11 +83,23 @@ public class ACastVisitor implements IVisitablePointableVisitor<Void, Triple<IVi
             caster = new AListCaster();
             laccessorToCaster.put(accessor, caster);
         }
-        if (arg.second.getTypeTag().equals(ATypeTag.ANY)) {
-            arg.second = accessor.ordered() ? DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE
-                    : DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
+
+        AbstractCollectionType resultType;
+        switch (arg.second.getTypeTag()) {
+            case ANY:
+                resultType = accessor.ordered() ? DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE
+                        : DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
+                break;
+            case ARRAY:
+            case MULTISET:
+                resultType = (AbstractCollectionType) arg.second;
+                break;
+            default:
+                throw new RuntimeDataException(ErrorCode.TYPE_CONVERT, sourceLoc,
+                        accessor.ordered() ? ATypeTag.ARRAY : ATypeTag.MULTISET, arg.second.getTypeTag());
         }
-        caster.castList(accessor, arg.first, (AbstractCollectionType) arg.second, this);
+
+        caster.castList(accessor, arg.first, resultType, this);
         return null;
     }
 
@@ -90,10 +111,20 @@ public class ACastVisitor implements IVisitablePointableVisitor<Void, Triple<IVi
             caster = new ARecordCaster();
             raccessorToCaster.put(accessor, caster);
         }
-        if (arg.second.getTypeTag().equals(ATypeTag.ANY)) {
-            arg.second = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+
+        ARecordType resultType;
+        switch (arg.second.getTypeTag()) {
+            case ANY:
+                resultType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+                break;
+            case OBJECT:
+                resultType = (ARecordType) arg.second;
+                break;
+            default:
+                throw new RuntimeDataException(ErrorCode.TYPE_CONVERT, sourceLoc, ATypeTag.OBJECT,
+                        arg.second.getTypeTag());
         }
-        ARecordType resultType = (ARecordType) arg.second;
+
         caster.castRecord(accessor, arg.first, resultType, this);
         return null;
     }
@@ -123,11 +154,11 @@ public class ACastVisitor implements IVisitablePointableVisitor<Void, Triple<IVi
                 ATypeHierarchy.convertNumericTypeByteArray(accessor.getByteArray(), accessor.getStartOffset(),
                         accessor.getLength(), reqTypeTag, castBuffer.getDataOutput(), strictDemote);
                 arg.first.set(castBuffer);
-            } catch (IOException e1) {
-                throw new HyracksDataException(
-                        "Type mismatch: cannot cast the " + inputTypeTag + " type to the " + reqTypeTag + " type.");
+            } catch (HyracksDataException e) {
+                throw e;
+            } catch (IOException e) {
+                throw new RuntimeDataException(ErrorCode.TYPE_CONVERT, sourceLoc, inputTypeTag, reqTypeTag);
             }
-
         }
 
         return null;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java
index f61518c..50c919b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java
@@ -35,17 +35,25 @@ public class TypeCastUtils {
 
     public static boolean setRequiredAndInputTypes(AbstractFunctionCallExpression expr, IAType requiredType,
             IAType inputType) throws CompilationException {
+        return setRequiredAndInputTypes(expr, requiredType, inputType, true);
+    }
+
+    public static boolean setRequiredAndInputTypes(AbstractFunctionCallExpression expr, IAType requiredType,
+            IAType inputType, boolean failIfTypeMismatch) throws CompilationException {
         boolean changed = false;
         Object[] opaqueParameters = expr.getOpaqueParameters();
         if (opaqueParameters == null) {
             opaqueParameters = new Object[2];
             opaqueParameters[0] = requiredType;
             opaqueParameters[1] = inputType;
-            ATypeTag requiredTypeTag = requiredType.getTypeTag();
-            ATypeTag actualTypeTag = TypeComputeUtils.getActualType(inputType).getTypeTag();
-            if (!ATypeHierarchy.isCompatible(requiredTypeTag, actualTypeTag)) {
-                FunctionIdentifier funcId = expr.getFunctionIdentifier();
-                throw new IncompatibleTypeException(expr.getSourceLocation(), funcId, actualTypeTag, requiredTypeTag);
+            if (failIfTypeMismatch) {
+                ATypeTag requiredTypeTag = requiredType.getTypeTag();
+                ATypeTag actualTypeTag = TypeComputeUtils.getActualType(inputType).getTypeTag();
+                if (!ATypeHierarchy.isCompatible(requiredTypeTag, actualTypeTag)) {
+                    FunctionIdentifier funcId = expr.getFunctionIdentifier();
+                    throw new IncompatibleTypeException(expr.getSourceLocation(), funcId, actualTypeTag,
+                            requiredTypeTag);
+                }
             }
             expr.setOpaqueParameters(opaqueParameters);
             changed = true;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
index 8080e16..c27423b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
@@ -88,7 +88,8 @@ public class CastTypeDescriptor extends AbstractScalarFunctionDynamicDescriptor
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
-                return new CastTypeEvaluator(reqType, inputType, recordEvalFactory.createScalarEvaluator(ctx));
+                IScalarEvaluator argEval = recordEvalFactory.createScalarEvaluator(ctx);
+                return new CastTypeEvaluator(reqType, inputType, argEval, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeEvaluator.java
index e8c2f6e..210149c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeEvaluator.java
@@ -29,6 +29,7 @@ import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -36,6 +37,7 @@ import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 public class CastTypeEvaluator implements IScalarEvaluator {
 
     private IScalarEvaluator argEvaluator;
+    protected final SourceLocation sourceLoc;
     private final IPointable argPointable = new VoidPointable();
     private final PointableAllocator allocator = new PointableAllocator();
     private IVisitablePointable inputPointable;
@@ -44,10 +46,22 @@ public class CastTypeEvaluator implements IScalarEvaluator {
     private final Triple<IVisitablePointable, IAType, Boolean> arg = new Triple<>(null, null, null);
 
     public CastTypeEvaluator() {
+        this(null);
+        // reset() should be called after using this constructor before calling any method
+    }
+
+    public CastTypeEvaluator(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
         // reset() should be called after using this constructor before calling any method
     }
 
     public CastTypeEvaluator(IAType reqType, IAType inputType, IScalarEvaluator argEvaluator) {
+        this(reqType, inputType, argEvaluator, null);
+    }
+
+    public CastTypeEvaluator(IAType reqType, IAType inputType, IScalarEvaluator argEvaluator,
+            SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
         resetAndAllocate(reqType, inputType, argEvaluator);
     }
 
@@ -61,7 +75,7 @@ public class CastTypeEvaluator implements IScalarEvaluator {
     }
 
     protected ACastVisitor createCastVisitor() {
-        return new ACastVisitor();
+        return new ACastVisitor(sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
index bdfe7a4..b23037c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxDescriptor.java
@@ -97,7 +97,8 @@ public class CastTypeLaxDescriptor extends AbstractScalarFunctionDynamicDescript
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
-                return new CastTypeLaxEvaluator(reqType, inputType, recordEvalFactory.createScalarEvaluator(ctx));
+                IScalarEvaluator argEval = recordEvalFactory.createScalarEvaluator(ctx);
+                return new CastTypeLaxEvaluator(reqType, inputType, argEval, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxEvaluator.java
index 35335c6..be74580 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeLaxEvaluator.java
@@ -24,6 +24,7 @@ import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -35,13 +36,13 @@ class CastTypeLaxEvaluator extends CastTypeEvaluator {
 
     private static final byte[] MISSING_BYTES = new byte[] { ATypeTag.SERIALIZED_MISSING_TYPE_TAG };
 
-    CastTypeLaxEvaluator(IAType reqType, IAType inputType, IScalarEvaluator argEvaluator) {
-        super(reqType, inputType, argEvaluator);
+    CastTypeLaxEvaluator(IAType reqType, IAType inputType, IScalarEvaluator argEvaluator, SourceLocation sourceLoc) {
+        super(reqType, inputType, argEvaluator, sourceLoc);
     }
 
     @Override
     protected ACastVisitor createCastVisitor() {
-        return new ACastVisitor(false);
+        return new ACastVisitor(false, sourceLoc);
     }
 
     @Override