Return-Path: X-Original-To: apmail-asterixdb-notifications-archive@minotaur.apache.org Delivered-To: apmail-asterixdb-notifications-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id C23CC18824 for ; Wed, 16 Mar 2016 21:02:50 +0000 (UTC) Received: (qmail 70276 invoked by uid 500); 16 Mar 2016 21:02:50 -0000 Delivered-To: apmail-asterixdb-notifications-archive@asterixdb.apache.org Received: (qmail 70245 invoked by uid 500); 16 Mar 2016 21:02:50 -0000 Mailing-List: contact notifications-help@asterixdb.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@asterixdb.incubator.apache.org Delivered-To: mailing list notifications@asterixdb.incubator.apache.org Received: (qmail 70236 invoked by uid 99); 16 Mar 2016 21:02:50 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 16 Mar 2016 21:02:50 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 33821C6A10 for ; Wed, 16 Mar 2016 21:02:50 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 2.126 X-Spam-Level: ** X-Spam-Status: No, score=2.126 tagged_above=-999 required=6.31 tests=[MISSING_HEADERS=1.207, SPF_FAIL=0.919] autolearn=disabled Received: from mx2-lw-us.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id dsjIvcL4q028 for ; Wed, 16 Mar 2016 21:02:46 +0000 (UTC) Received: from unhygienix.ics.uci.edu (unhygienix.ics.uci.edu [128.195.14.130]) by mx2-lw-us.apache.org (ASF Mail Server at mx2-lw-us.apache.org) with ESMTP id 354D35FB3F for ; Wed, 16 Mar 2016 21:02:45 +0000 (UTC) Received: from localhost (localhost [127.0.0.1]) by unhygienix.ics.uci.edu (Postfix) with ESMTP id 06F9724195B; Wed, 16 Mar 2016 13:57:55 -0700 (PDT) Date: Wed, 16 Mar 2016 13:57:54 -0700 From: "Yingyi Bu (Code Review)" CC: Jenkins , Till Westmann , Preston Carman , Wail Alkowaileet Reply-To: buyingyi@gmail.com X-Gerrit-MessageType: merged Subject: Change in asterixdb[master]: ASTERIXDB-865: fix query compilation for if-else expression. X-Gerrit-Change-Id: I80e7995e814180fe567818263c79493901af284c X-Gerrit-ChangeURL: X-Gerrit-Commit: 22ad8e732e870f3caeae9ec30f5a2ad4c8732bcb In-Reply-To: References: MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Content-Disposition: inline User-Agent: Gerrit/2.8.4 Message-Id: <20160316205755.06F9724195B@unhygienix.ics.uci.edu> Yingyi Bu has submitted this change and it was merged. Change subject: ASTERIXDB-865: fix query compilation for if-else expression. ...................................................................... ASTERIXDB-865: fix query compilation for if-else expression. Change-Id: I80e7995e814180fe567818263c79493901af284c Reviewed-on: https://asterix-gerrit.ics.uci.edu/703 Tested-by: Jenkins Reviewed-by: Till Westmann --- M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java M asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java A asterix-app/data/page_views.adm A asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1203.aql A asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-865.aql A asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1203.plan A asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-865.plan M asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1308-1.plan M asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1308-2.plan A asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.1.ddl.aql A asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.2.update.aql A asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.3.query.aql A asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.1.ddl.aql A asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.2.update.aql A asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.3.query.aql A asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.1.adm A asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.1.adm M asterix-app/src/test/resources/runtimets/testsuite.xml M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeCompatibilityChecker.java 22 files changed, 550 insertions(+), 102 deletions(-) Approvals: Till Westmann: Looks good to me, approved Jenkins: Verified diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java index 3eadba3..cda7619 100644 --- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java +++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java @@ -19,12 +19,11 @@ package org.apache.asterix.optimizer.rules; -import org.apache.commons.lang3.mutable.Mutable; - import org.apache.asterix.om.types.ATypeTag; import org.apache.asterix.om.types.AUnionType; import org.apache.asterix.om.types.IAType; import org.apache.asterix.om.util.NonTaggedFormatUtil; +import org.apache.commons.lang3.mutable.Mutable; import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException; import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression; import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator; @@ -45,7 +44,8 @@ public class CheckFilterExpressionTypeRule implements IAlgebraicRewriteRule { @Override - public boolean rewritePre(Mutable opRef, IOptimizationContext context) throws AlgebricksException { + public boolean rewritePre(Mutable opRef, IOptimizationContext context) + throws AlgebricksException { return false; } @@ -62,8 +62,8 @@ IAType condType = (IAType) env.getType(condition); if (condType.getTypeTag() != ATypeTag.BOOLEAN && condType.getTypeTag() != ATypeTag.ANY && !isPossibleBoolean(condType)) { - throw new AlgebricksException("The select condition " + condition.toString() - + " should be of the boolean type."); + throw new AlgebricksException( + "The select condition " + condition.toString() + " should be of the boolean type."); } return false; } diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java index a9b8e99..58921b4 100644 --- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java +++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java @@ -63,6 +63,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator; import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator; import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities; +import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil; import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule; public class PushFieldAccessRule implements IAlgebraicRewriteRule { @@ -190,6 +191,9 @@ && !(op2.getOperatorTag() == LogicalOperatorTag.SELECT && isAccessToIndexedField(access, context))) { return false; } + if (!OperatorPropertiesUtil.isMovable(op2)) { + return false; + } if (tryingToPushThroughSelectionWithSameDataSource(access, op2)) { return false; } diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java index 8313504..d5a6f2b 100644 --- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java +++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java @@ -99,7 +99,7 @@ // Match (assign | unnest)+. while ((subTreeOp.getOperatorTag() == LogicalOperatorTag.ASSIGN || subTreeOp.getOperatorTag() == LogicalOperatorTag.UNNEST)) { - if (OperatorPropertiesUtil.isStatefulAssign(subTreeOp)) { + if (!OperatorPropertiesUtil.isMovable(subTreeOp)) { return false; } else { assignsAndUnnestsRefs.add(subTreeOpRef); diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java index 28f1fca..0ec12e5 100644 --- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java +++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java @@ -51,6 +51,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.PrimaryKeyVariablesVisitor; import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities; import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency; +import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil; import org.apache.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil; import org.mortbay.util.SingletonList; @@ -173,11 +174,9 @@ } else { LogicalVariable assignVar = context.newVar(); ILogicalOperator assignOp = new AssignOperator(assignVar, - new MutableObject(usedForCorrelationJoin - ? new StatefulFunctionCallExpression( - FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.CREATE_QUERY_UID), null) - : new ScalarFunctionCallExpression( - FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.CREATE_QUERY_UID)))); + new MutableObject(new StatefulFunctionCallExpression( + FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.CREATE_QUERY_UID), null))); + OperatorPropertiesUtil.markMovable(assignOp, !usedForCorrelationJoin); assignOp.getInputs().add(new MutableObject(operator)); context.addPrimaryKey(new FunctionalDependency(Collections.singletonList(assignVar), new ArrayList(liveVars))); diff --git a/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java index bd0694a..10f70f1 100644 --- a/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java +++ b/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java @@ -21,6 +21,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -144,6 +145,7 @@ import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain; import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty; import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn; +import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil; import org.apache.hyracks.api.io.FileReference; import org.apache.hyracks.dataflow.std.file.FileSplit; @@ -783,50 +785,49 @@ // on top of which there is a selection whose condition is varCond. // Similarly, we create one subplan for the "else" branch, in which the // selection is not(varCond). - // Finally, we concatenate the results. (??) - + // Finally, we select the desired result. Pair pCond = ifexpr.getCondExpr().accept(this, tupSource); - ILogicalOperator opCond = pCond.first; LogicalVariable varCond = pCond.second; - SubplanOperator sp = new SubplanOperator(); + //Creates a subplan for the "then" branch. + Pair opAndVarForThen = constructSubplanOperatorForBranch(pCond.first, + new MutableObject(new VariableReferenceExpression(varCond)), ifexpr.getThenExpr()); - Mutable nestedSource = new MutableObject( - new NestedTupleSourceOperator(new MutableObject(sp))); - - // Enters/exists subplan for the then-expr and the else-expr respectively. - context.enterSubplan(); - Pair pThen = ifexpr.getThenExpr().accept(this, nestedSource); - SelectOperator sel1 = new SelectOperator( - new MutableObject(new VariableReferenceExpression(varCond)), false, null); - sel1.getInputs().add(new MutableObject(pThen.first)); - context.exitSubplan(); - - context.enterSubplan(); - Pair pElse = ifexpr.getElseExpr().accept(this, nestedSource); + // Creates a subplan for the "else" branch. AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression( - FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), - new MutableObject(new VariableReferenceExpression(varCond))); - SelectOperator sel2 = new SelectOperator(new MutableObject(notVarCond), false, null); - sel2.getInputs().add(new MutableObject(pElse.first)); - context.exitSubplan(); + FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), Collections.singletonList( + new MutableObject(new VariableReferenceExpression(varCond)))); + Pair opAndVarForElse = constructSubplanOperatorForBranch( + opAndVarForThen.first, new MutableObject(notVarCond), ifexpr.getElseExpr()); - ILogicalPlan p1 = new ALogicalPlanImpl(new MutableObject(sel1)); - sp.getNestedPlans().add(p1); - ILogicalPlan p2 = new ALogicalPlanImpl(new MutableObject(sel2)); - sp.getNestedPlans().add(p2); + // Uses switch-case function to select the results of two branches. + LogicalVariable selectVar = context.newVar(); + List> arguments = new ArrayList<>(); + arguments.add(new MutableObject(new VariableReferenceExpression(varCond))); + arguments.add(new MutableObject(ConstantExpression.TRUE)); + arguments.add(new MutableObject(new VariableReferenceExpression(opAndVarForThen.second))); + arguments.add(new MutableObject(ConstantExpression.FALSE)); + arguments.add(new MutableObject(new VariableReferenceExpression(opAndVarForElse.second))); + AbstractFunctionCallExpression swithCaseExpr = new ScalarFunctionCallExpression( + FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.SWITCH_CASE), arguments); + AssignOperator assignOp = new AssignOperator(selectVar, new MutableObject(swithCaseExpr)); + assignOp.getInputs().add(new MutableObject(opAndVarForElse.first)); - Mutable opCondRef = new MutableObject(opCond); - sp.getInputs().add(opCondRef); + // Unnests the selected ("if" or "else") result. + LogicalVariable unnestVar = context.newVar(); + UnnestOperator unnestOp = new UnnestOperator(unnestVar, + new MutableObject(new UnnestingFunctionCallExpression( + FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION), + Collections.singletonList( + new MutableObject(new VariableReferenceExpression(selectVar)))))); + unnestOp.getInputs().add(new MutableObject(assignOp)); - LogicalVariable resV = context.newVar(); - AbstractFunctionCallExpression concatNonNull = new ScalarFunctionCallExpression( - FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.CONCAT_NON_NULL), - new MutableObject(new VariableReferenceExpression(pThen.second)), - new MutableObject(new VariableReferenceExpression(pElse.second))); - AssignOperator a = new AssignOperator(resV, new MutableObject(concatNonNull)); - a.getInputs().add(new MutableObject(sp)); - return new Pair(a, resV); + // Produces the final result. + LogicalVariable resultVar = context.newVar(); + AssignOperator finalAssignOp = new AssignOperator(resultVar, + new MutableObject(new VariableReferenceExpression(unnestVar))); + finalAssignOp.getInputs().add(new MutableObject(unnestOp)); + return new Pair(finalAssignOp, resultVar); } @Override @@ -1307,7 +1308,7 @@ return (k == Kind.LITERAL_EXPRESSION) || (k == Kind.LIST_CONSTRUCTOR_EXPRESSION) || (k == Kind.RECORD_CONSTRUCTOR_EXPRESSION) || (k == Kind.VARIABLE_EXPRESSION) || (k == Kind.CALL_EXPRESSION) || (k == Kind.OP_EXPRESSION) || (k == Kind.FIELD_ACCESSOR_EXPRESSION) - || (k == Kind.INDEX_ACCESSOR_EXPRESSION) || (k == Kind.UNARY_EXPRESSION); + || (k == Kind.INDEX_ACCESSOR_EXPRESSION) || (k == Kind.UNARY_EXPRESSION) || (k == Kind.IF_EXPRESSION); } protected List mkSingletonArrayList(T item) { @@ -1451,4 +1452,42 @@ throw new AsterixException(e); } } + + /** + * Constructs a subplan operator for a branch in a if-else (or case) expression. + * + * @param inputOp, + * the input operator. + * @param selectExpr, + * the expression to select tuples that are processed by this branch. + * @param branchExpression, + * the expression to be evaluated in this branch. + * @return a pair of the constructed subplan operator and the output variable for the branch. + * @throws AsterixException + */ + private Pair constructSubplanOperatorForBranch(ILogicalOperator inputOp, + Mutable selectExpr, Expression branchExpression) throws AsterixException { + context.enterSubplan(); + SubplanOperator subplanOp = new SubplanOperator(); + subplanOp.getInputs().add(new MutableObject(inputOp)); + Mutable nestedSource = new MutableObject( + new NestedTupleSourceOperator(new MutableObject(subplanOp))); + SelectOperator select = new SelectOperator(selectExpr, false, null); + // The select operator cannot be moved up and down, otherwise it will cause typing issues (ASTERIXDB-1203). + OperatorPropertiesUtil.markMovable(select, false); + select.getInputs().add(nestedSource); + Pair pBranch = branchExpression.accept(this, + new MutableObject(select)); + LogicalVariable branchVar = context.newVar(); + AggregateOperator aggOp = new AggregateOperator(Collections.singletonList(branchVar), + Collections.singletonList(new MutableObject(new AggregateFunctionCallExpression( + FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.LISTIFY), false, + Collections.singletonList(new MutableObject( + new VariableReferenceExpression(pBranch.second))))))); + aggOp.getInputs().add(new MutableObject(pBranch.first)); + ILogicalPlan planForBranch = new ALogicalPlanImpl(new MutableObject(aggOp)); + subplanOp.getNestedPlans().add(planForBranch); + context.exitSubplan(); + return new Pair(subplanOp, branchVar); + } } diff --git a/asterix-app/data/page_views.adm b/asterix-app/data/page_views.adm new file mode 100644 index 0000000..f09e52d --- /dev/null +++ b/asterix-app/data/page_views.adm @@ -0,0 +1,9 @@ +{"user":"John", "action": 1, "timespent": 2, "query_term":"qt", "ip_addr": 3, "timestamp": 4, "estimated_revenue": 5.0, +"page_info" : { "a":"aaa","b":"bbb" }, +"page_links": {{ {"b":"ccc","d":"ddd","e":"eee"},{"b":"fff","g":"ggg","h":"hhh"} }} +} +{"user":"Bill", "action": 2, "timespent": 2, "query_term":"qt", "ip_addr": 3, "timestamp": 4, "estimated_revenue": 5.0, +"page_info" : { "a":"aaa","b":"bbb" }, +"page_links": {{ {"c":"ccc","d":"ddd","a":"eee"},{"f":"fff","g":"ggg","b":"bbb"} }} +} + diff --git a/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1203.aql b/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1203.aql new file mode 100644 index 0000000..c3944b1 --- /dev/null +++ b/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1203.aql @@ -0,0 +1,41 @@ +/* + * 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. + */ + +/* This test is to verify the fix for ASTERIXDB-1203. */ + +drop dataverse test if exists; +create dataverse test; +use dataverse test; + +create type ifType as open +{ + id: int32 +} + +create dataset ifds(ifType) +primary key id; + +for $x in dataset ifds +where ( +if ($x.names.count = "1") then + $x.names.name.firstName = "Tom" +else + (some $v in $x.names.name satisfies $v.firstName = "Tom") +) +return $x; diff --git a/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-865.aql b/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-865.aql new file mode 100644 index 0000000..185e74b --- /dev/null +++ b/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-865.aql @@ -0,0 +1,52 @@ +/* + * 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. + */ + +/* This test is to verify the fix for ASTERIXDB-865. */ + +drop dataverse test if exists; +create dataverse test; +use dataverse test; + +create type page_info_type as open {} + +create type page_views_type as closed { + user: string, + action: int32, + timespent: int32, + query_term: string, + ip_addr: int32, + timestamp: int32, + estimated_revenue: double, + page_info: page_info_type, + page_links: {{ page_info_type}} +} + +create dataset page_views(page_views_type) +primary key user; + +for $t in dataset page_views +let $header := if ($t.action = 1) + then [ $t.page_info.a ] + else + for $pl in $t.page_links + return $pl.b +for $h in $header +return +{ "user": $t.user, "header": $h } +; diff --git a/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1203.plan b/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1203.plan new file mode 100644 index 0000000..9dc7a8a --- /dev/null +++ b/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1203.plan @@ -0,0 +1,36 @@ +-- DISTRIBUTE_RESULT |PARTITIONED| + -- ONE_TO_ONE_EXCHANGE |PARTITIONED| + -- STREAM_PROJECT |PARTITIONED| + -- STREAM_SELECT |PARTITIONED| + -- STREAM_PROJECT |PARTITIONED| + -- UNNEST |PARTITIONED| + -- STREAM_PROJECT |PARTITIONED| + -- ASSIGN |PARTITIONED| + -- SUBPLAN |PARTITIONED| + { + -- AGGREGATE |LOCAL| + -- AGGREGATE |LOCAL| + -- STREAM_SELECT |LOCAL| + -- ASSIGN |LOCAL| + -- UNNEST |LOCAL| + -- ASSIGN |LOCAL| + -- ASSIGN |LOCAL| + -- STREAM_SELECT |LOCAL| + -- NESTED_TUPLE_SOURCE |LOCAL| + } + -- SUBPLAN |PARTITIONED| + { + -- AGGREGATE |LOCAL| + -- ASSIGN |LOCAL| + -- ASSIGN |LOCAL| + -- ASSIGN |LOCAL| + -- ASSIGN |LOCAL| + -- STREAM_SELECT |LOCAL| + -- NESTED_TUPLE_SOURCE |LOCAL| + } + -- ASSIGN |PARTITIONED| + -- STREAM_PROJECT |PARTITIONED| + -- ONE_TO_ONE_EXCHANGE |PARTITIONED| + -- DATASOURCE_SCAN |PARTITIONED| + -- ONE_TO_ONE_EXCHANGE |PARTITIONED| + -- EMPTY_TUPLE_SOURCE |PARTITIONED| diff --git a/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-865.plan b/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-865.plan new file mode 100644 index 0000000..6ff5cee --- /dev/null +++ b/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-865.plan @@ -0,0 +1,35 @@ +-- DISTRIBUTE_RESULT |PARTITIONED| + -- ONE_TO_ONE_EXCHANGE |PARTITIONED| + -- STREAM_PROJECT |PARTITIONED| + -- ASSIGN |PARTITIONED| + -- STREAM_PROJECT |PARTITIONED| + -- UNNEST |PARTITIONED| + -- STREAM_PROJECT |PARTITIONED| + -- UNNEST |PARTITIONED| + -- STREAM_PROJECT |PARTITIONED| + -- ASSIGN |PARTITIONED| + -- STREAM_PROJECT |PARTITIONED| + -- SUBPLAN |PARTITIONED| + { + -- AGGREGATE |LOCAL| + -- AGGREGATE |LOCAL| + -- ASSIGN |LOCAL| + -- UNNEST |LOCAL| + -- ASSIGN |LOCAL| + -- STREAM_SELECT |LOCAL| + -- NESTED_TUPLE_SOURCE |LOCAL| + } + -- SUBPLAN |PARTITIONED| + { + -- AGGREGATE |LOCAL| + -- ASSIGN |LOCAL| + -- ASSIGN |LOCAL| + -- ASSIGN |LOCAL| + -- STREAM_SELECT |LOCAL| + -- NESTED_TUPLE_SOURCE |LOCAL| + } + -- ASSIGN |PARTITIONED| + -- ONE_TO_ONE_EXCHANGE |PARTITIONED| + -- DATASOURCE_SCAN |PARTITIONED| + -- ONE_TO_ONE_EXCHANGE |PARTITIONED| + -- EMPTY_TUPLE_SOURCE |PARTITIONED| diff --git a/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1308-1.plan b/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1308-1.plan index ebbee79..3664ed8 100644 --- a/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1308-1.plan +++ b/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1308-1.plan @@ -23,23 +23,25 @@ -- ASSIGN |UNPARTITIONED| -- STREAM_PROJECT |UNPARTITIONED| -- UNNEST |UNPARTITIONED| - -- SUBPLAN |UNPARTITIONED| - { - -- SUBPLAN |UNPARTITIONED| - { - -- AGGREGATE |UNPARTITIONED| - -- STREAM_SELECT |UNPARTITIONED| - -- UNNEST |UNPARTITIONED| - -- SUBPLAN |UNPARTITIONED| - { - -- AGGREGATE |UNPARTITIONED| - -- IN_MEMORY_STABLE_SORT [$$21(ASC)] |UNPARTITIONED| - -- UNNEST |UNPARTITIONED| - -- NESTED_TUPLE_SOURCE |UNPARTITIONED| - } - -- NESTED_TUPLE_SOURCE |UNPARTITIONED| - } - -- NESTED_TUPLE_SOURCE |UNPARTITIONED| - } - -- ASSIGN |UNPARTITIONED| - -- EMPTY_TUPLE_SOURCE |UNPARTITIONED| + -- STREAM_PROJECT |UNPARTITIONED| + -- SUBPLAN |UNPARTITIONED| + { + -- ASSIGN |UNPARTITIONED| + -- SUBPLAN |UNPARTITIONED| + { + -- AGGREGATE |UNPARTITIONED| + -- STREAM_SELECT |UNPARTITIONED| + -- UNNEST |UNPARTITIONED| + -- SUBPLAN |UNPARTITIONED| + { + -- AGGREGATE |UNPARTITIONED| + -- IN_MEMORY_STABLE_SORT [$$21(ASC)] |UNPARTITIONED| + -- UNNEST |UNPARTITIONED| + -- NESTED_TUPLE_SOURCE |UNPARTITIONED| + } + -- NESTED_TUPLE_SOURCE |UNPARTITIONED| + } + -- NESTED_TUPLE_SOURCE |UNPARTITIONED| + } + -- ASSIGN |UNPARTITIONED| + -- EMPTY_TUPLE_SOURCE |UNPARTITIONED| diff --git a/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1308-2.plan b/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1308-2.plan index a6e7496..caf4c48 100644 --- a/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1308-2.plan +++ b/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1308-2.plan @@ -9,22 +9,23 @@ { -- SUBPLAN |LOCAL| { - -- SUBPLAN |LOCAL| - { - -- AGGREGATE |LOCAL| - -- STREAM_SELECT |LOCAL| - -- ASSIGN |LOCAL| - -- UNNEST |LOCAL| - -- SUBPLAN |LOCAL| - { - -- AGGREGATE |LOCAL| - -- IN_MEMORY_STABLE_SORT [$$38(ASC)] |LOCAL| - -- UNNEST |LOCAL| - -- NESTED_TUPLE_SOURCE |LOCAL| - } - -- NESTED_TUPLE_SOURCE |LOCAL| - } - -- NESTED_TUPLE_SOURCE |LOCAL| + -- ASSIGN |LOCAL| + -- SUBPLAN |LOCAL| + { + -- AGGREGATE |LOCAL| + -- STREAM_SELECT |LOCAL| + -- ASSIGN |LOCAL| + -- UNNEST |LOCAL| + -- SUBPLAN |LOCAL| + { + -- AGGREGATE |LOCAL| + -- IN_MEMORY_STABLE_SORT [$$38(ASC)] |LOCAL| + -- UNNEST |LOCAL| + -- NESTED_TUPLE_SOURCE |LOCAL| + } + -- NESTED_TUPLE_SOURCE |LOCAL| + } + -- NESTED_TUPLE_SOURCE |LOCAL| } -- SUBPLAN |LOCAL| { @@ -38,22 +39,23 @@ } -- SUBPLAN |LOCAL| { - -- SUBPLAN |LOCAL| - { - -- AGGREGATE |LOCAL| - -- STREAM_SELECT |LOCAL| - -- ASSIGN |LOCAL| - -- UNNEST |LOCAL| - -- SUBPLAN |LOCAL| - { - -- AGGREGATE |LOCAL| - -- IN_MEMORY_STABLE_SORT [$$31(ASC)] |LOCAL| - -- UNNEST |LOCAL| - -- NESTED_TUPLE_SOURCE |LOCAL| - } - -- NESTED_TUPLE_SOURCE |LOCAL| - } - -- NESTED_TUPLE_SOURCE |LOCAL| + -- ASSIGN |LOCAL| + -- SUBPLAN |LOCAL| + { + -- AGGREGATE |LOCAL| + -- STREAM_SELECT |LOCAL| + -- ASSIGN |LOCAL| + -- UNNEST |LOCAL| + -- SUBPLAN |LOCAL| + { + -- AGGREGATE |LOCAL| + -- IN_MEMORY_STABLE_SORT [$$31(ASC)] |LOCAL| + -- UNNEST |LOCAL| + -- NESTED_TUPLE_SOURCE |LOCAL| + } + -- NESTED_TUPLE_SOURCE |LOCAL| + } + -- NESTED_TUPLE_SOURCE |LOCAL| } -- NESTED_TUPLE_SOURCE |LOCAL| } diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.1.ddl.aql new file mode 100644 index 0000000..44468b1 --- /dev/null +++ b/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.1.ddl.aql @@ -0,0 +1,32 @@ +/* + * 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. + */ + + +drop dataverse test if exists; +create dataverse test; +use dataverse test; + +create type ifType as open +{ + id: int32 +} + +create dataset ifds(ifType) +primary key id; + diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.2.update.aql new file mode 100644 index 0000000..66817ec --- /dev/null +++ b/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.2.update.aql @@ -0,0 +1,52 @@ +/* + * 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. + */ + +use dataverse test; + +insert into dataset ifds( +{ + "id":0, + "names":{ + "count": "1", + "name" :{ + "firstName" : "Tom", + "lastName" : "Smith" + } + } +} +) + +insert into dataset ifds( +{ + "id":1, + "names":{ + "count": "2", + "name" :[ + { + "firstName" : "Tom", + "lastName" : "Smith" + }, + { + "firstName" : "Bob", + "lastName" : "Jones" + } + ] + } +} +) diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.3.query.aql new file mode 100644 index 0000000..9a27a1d --- /dev/null +++ b/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.3.query.aql @@ -0,0 +1,31 @@ +/* + * 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. + */ + +/* This test is to verify the fix for ASTERIXDB-1203. */ + +use dataverse test; + +for $x in dataset ifds +where ( +if ($x.names.count = "1") then + $x.names.name.firstName = "Tom" +else + (some $v in $x.names.name satisfies $v.firstName = "Tom") +) +return $x; diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.1.ddl.aql new file mode 100644 index 0000000..bbb9c2a --- /dev/null +++ b/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.1.ddl.aql @@ -0,0 +1,40 @@ +/* + * 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. + */ + + +drop dataverse test if exists; +create dataverse test; +use dataverse test; + +create type page_info_type as open {} + +create type page_views_type as closed { + user: string, + action: int32, + timespent: int32, + query_term: string, + ip_addr: int32, + timestamp: int32, + estimated_revenue: double, + page_info: page_info_type, + page_links: {{ page_info_type}} +} + +create dataset page_views(page_views_type) +primary key user; diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.2.update.aql new file mode 100644 index 0000000..e42334b --- /dev/null +++ b/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.2.update.aql @@ -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. + */ + +use dataverse test; + +load dataset page_views using localfs +(("path"="asterix_nc1://data/page_views.adm"),("format"="adm")); diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.3.query.aql new file mode 100644 index 0000000..f7165ed --- /dev/null +++ b/asterix-app/src/test/resources/runtimets/queries/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.3.query.aql @@ -0,0 +1,35 @@ +/* + * 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. + */ + +/* This test is to verify the fix for ASTERIXDB-865. */ + +use dataverse test; + +for $t in dataset page_views +let $header := if ($t.action = 1) + then [ $t.page_info.a ] + else + for $pl in $t.page_links + return $pl.b +for $h in $header +order by $t.user, $h +return +{ "user": $t.user, "header": $h } +; + diff --git a/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.1.adm b/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.1.adm new file mode 100644 index 0000000..373b4c7 --- /dev/null +++ b/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-1203/query-ASTERIXDB-1203.1.adm @@ -0,0 +1,2 @@ +{ "id": 1i32, "names": { "count": "2", "name": [ { "firstName": "Tom", "lastName": "Smith" }, { "firstName": "Bob", "lastName": "Jones" } ] } } +{ "id": 0i32, "names": { "count": "1", "name": { "firstName": "Tom", "lastName": "Smith" } } } diff --git a/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.1.adm b/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.1.adm new file mode 100644 index 0000000..204e1b6 --- /dev/null +++ b/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-865/query-ASTERIXDB-865.1.adm @@ -0,0 +1,3 @@ +{ "user": "Bill", "header": null } +{ "user": "Bill", "header": "bbb" } +{ "user": "John", "header": "aaa" } diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml index 0ac4a1e..9dcd883 100644 --- a/asterix-app/src/test/resources/runtimets/testsuite.xml +++ b/asterix-app/src/test/resources/runtimets/testsuite.xml @@ -3058,6 +3058,16 @@ prefix-search + + + query-ASTERIXDB-865 + + + + + query-ASTERIXDB-1203 + + diff --git a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeCompatibilityChecker.java b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeCompatibilityChecker.java index 9d2c2ed..565ed02 100644 --- a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeCompatibilityChecker.java +++ b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeCompatibilityChecker.java @@ -47,8 +47,9 @@ for (IAType t : typeList) { if (t.getTypeTag() != ATypeTag.NULL) { //CONCAT_NON_NULL cannot return null because it's only used for if-else construct - if (!possibleTypes.contains(t)) + if (!possibleTypes.contains(t)) { possibleTypes.add(t); + } } else { nullEncountered = true; } @@ -75,6 +76,6 @@ return possibleTypes.get(0); } } - return null; + return BuiltinType.ANY; } } -- To view, visit https://asterix-gerrit.ics.uci.edu/703 To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings Gerrit-MessageType: merged Gerrit-Change-Id: I80e7995e814180fe567818263c79493901af284c Gerrit-PatchSet: 12 Gerrit-Project: asterixdb Gerrit-Branch: master Gerrit-Owner: Yingyi Bu Gerrit-Reviewer: Jenkins Gerrit-Reviewer: Preston Carman Gerrit-Reviewer: Till Westmann Gerrit-Reviewer: Wail Alkowaileet Gerrit-Reviewer: Yingyi Bu