You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by bu...@apache.org on 2016/02/12 19:47:56 UTC

incubator-asterixdb git commit: Fix ASTERIXDB-1298

Repository: incubator-asterixdb
Updated Branches:
  refs/heads/master 760ce9041 -> f93405cd5


Fix ASTERIXDB-1298

Handle the case that the original join operator in a subplan
operator is a left-outer join such that no null-check
variable needs to be added.

Change-Id: Ibb594e161d7236131a7a6a0a290e76071d4587f4
Reviewed-on: https://asterix-gerrit.ics.uci.edu/630
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Steven Jacobs <sj...@ucr.edu>


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

Branch: refs/heads/master
Commit: f93405cd5fddc3b14b05f4096d8d82f4c907e8f9
Parents: 760ce90
Author: Yingyi Bu <yi...@couchbase.com>
Authored: Fri Feb 12 09:07:17 2016 -0800
Committer: Yingyi Bu <bu...@gmail.com>
Committed: Fri Feb 12 10:43:13 2016 -0800

----------------------------------------------------------------------
 ...ineSubplanInputForNestedTupleSourceRule.java | 49 +++++++++--------
 .../queries/udfs/query-ASTERIXDB-1298.aql       | 49 +++++++++++++++++
 .../results/udfs/query-ASTERIXDB-1298.plan      | 20 +++++++
 .../query-ASTERIXDB-1298.1.ddl.aql              | 41 ++++++++++++++
 .../query-ASTERIXDB-1298.2.update.aql           | 56 ++++++++++++++++++++
 .../query-ASTERIXDB-1298.3.query.aql            | 25 +++++++++
 .../query-ASTERIXDB-1298.1.adm                  |  4 ++
 .../src/test/resources/runtimets/testsuite.xml  |  5 ++
 8 files changed, 227 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f93405cd/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
index d88b69f..9f03ffa 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
@@ -503,29 +503,34 @@ public class InlineSubplanInputForNestedTupleSourceRule implements IAlgebraicRew
         }
         groupbyOp.getInputs().add(new MutableObject<ILogicalOperator>(topJoinRef.getValue()));
 
-        // Adds a select operator into the nested plan for group-by to remove tuples with NULL on {@code assignVar}, i.e.,
-        // subplan input tuples that are filtered out within a subplan.
-        List<Mutable<ILogicalExpression>> nullCheckExprRefs = new ArrayList<>();
-        for (LogicalVariable notNullVar : notNullVars) {
-            Mutable<ILogicalExpression> filterVarExpr = new MutableObject<ILogicalExpression>(
-                    new VariableReferenceExpression(notNullVar));
-            List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
-            args.add(filterVarExpr);
-            List<Mutable<ILogicalExpression>> argsForNotFunction = new ArrayList<Mutable<ILogicalExpression>>();
-            argsForNotFunction.add(new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
-                    FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.IS_NULL), args)));
-            nullCheckExprRefs.add(new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
-                    FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.NOT), argsForNotFunction)));
+        if (notNullVars.size() > 0) {
+            // Adds a select operator into the nested plan for group-by to remove tuples with NULL on {@code assignVar}, i.e.,
+            // subplan input tuples that are filtered out within a subplan.
+            List<Mutable<ILogicalExpression>> nullCheckExprRefs = new ArrayList<>();
+            for (LogicalVariable notNullVar : notNullVars) {
+                Mutable<ILogicalExpression> filterVarExpr = new MutableObject<ILogicalExpression>(
+                        new VariableReferenceExpression(notNullVar));
+                List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
+                args.add(filterVarExpr);
+                List<Mutable<ILogicalExpression>> argsForNotFunction = new ArrayList<Mutable<ILogicalExpression>>();
+                argsForNotFunction.add(new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
+                        FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.IS_NULL), args)));
+                nullCheckExprRefs.add(new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
+                        FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.NOT), argsForNotFunction)));
+            }
+            Mutable<ILogicalExpression> selectExprRef = nullCheckExprRefs.size() > 1
+                    ? new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
+                            FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.AND), nullCheckExprRefs))
+                    : nullCheckExprRefs.get(0);
+            SelectOperator selectOp = new SelectOperator(selectExprRef, false, null);
+            topJoinRef.setValue(selectOp);
+            selectOp.getInputs().add(new MutableObject<ILogicalOperator>(
+                    new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(groupbyOp))));
+        } else {
+            // The original join operator in the Subplan is a left-outer join.
+            // Therefore, no null-check variable is injected and no SelectOperator needs to be added.
+            topJoinRef.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(groupbyOp)));
         }
-        Mutable<ILogicalExpression> selectExprRef = nullCheckExprRefs.size() > 1
-                ? new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.AND), nullCheckExprRefs))
-                : nullCheckExprRefs.get(0);
-        SelectOperator selectOp = new SelectOperator(selectExprRef, false, null);
-        topJoinRef.setValue(selectOp);
-        selectOp.getInputs().add(new MutableObject<ILogicalOperator>(
-                new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(groupbyOp))));
-
         opRef.setValue(groupbyOp);
         OperatorManipulationUtil.computeTypeEnvironmentBottomUp(groupbyOp, context);
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f93405cd/asterix-app/src/test/resources/optimizerts/queries/udfs/query-ASTERIXDB-1298.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/optimizerts/queries/udfs/query-ASTERIXDB-1298.aql b/asterix-app/src/test/resources/optimizerts/queries/udfs/query-ASTERIXDB-1298.aql
new file mode 100644
index 0000000..996ba05
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/udfs/query-ASTERIXDB-1298.aql
@@ -0,0 +1,49 @@
+/*
+ * 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 emergencyTest if exists;
+create dataverse emergencyTest;
+use dataverse emergencyTest;
+
+
+drop dataverse channels if exists;
+create dataverse channels;
+use dataverse channels;
+
+create type userLocation as closed
+{ userId: int, roomNumber: int }
+
+create dataset userLocations(userLocation)
+primary key userId;
+
+create function currentOccupancy($room)
+{
+ let $list := for $location in dataset userLocations
+ where $location.roomNumber = $room
+ return $location.userId return $list
+};
+
+create type subscription as { "id":uuid, "param0":int }
+
+create dataset subscriptions(subscription)
+primary key id autogenerated;
+
+for $sub in dataset subscriptions
+for $result in currentOccupancy($sub.param0)
+return $result;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f93405cd/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1298.plan
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1298.plan b/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1298.plan
new file mode 100644
index 0000000..c62d34e
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/udfs/query-ASTERIXDB-1298.plan
@@ -0,0 +1,20 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- HYBRID_HASH_JOIN [$$5][$$17]  |PARTITIONED|
+          -- HASH_PARTITION_EXCHANGE [$$5]  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- HASH_PARTITION_EXCHANGE [$$17]  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f93405cd/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.1.ddl.aql
new file mode 100644
index 0000000..966808d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.1.ddl.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.
+ */
+
+
+drop dataverse channels if exists;
+create dataverse channels;
+use dataverse channels;
+
+create type userLocation as closed
+{ userId: int, roomNumber: int }
+
+create dataset userLocations(userLocation)
+primary key userId;
+
+create function currentOccupancy($room)
+{
+ let $list := for $location in dataset userLocations
+ where $location.roomNumber = $room
+ return $location.userId return $list
+};
+
+create type subscription as { "id":uuid, "param0":int }
+
+create dataset subscriptions(subscription)
+primary key id autogenerated;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f93405cd/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.2.update.aql
new file mode 100644
index 0000000..b715a6e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.2.update.aql
@@ -0,0 +1,56 @@
+/*
+ * 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 channels;
+
+upsert into dataset userLocations(
+[
+{"userId":1, "roomNumber":123}
+,
+{"userId":2, "roomNumber":123}
+,
+{"userId":3, "roomNumber":123}
+,
+{"userId":4, "roomNumber":123}
+,
+{"userId":5, "roomNumber":350}
+,
+{"userId":6, "roomNumber":350}
+,
+{"userId":7, "roomNumber":350}
+,
+{"userId":8, "roomNumber":350}
+,
+{"userId":9, "roomNumber":350}
+,
+{"userId":10,"roomNumber":210}
+,
+{"userId":11,"roomNumber":210}
+,
+{"userId":12,"roomNumber":210}
+,
+{"userId":13,"roomNumber":210}
+,
+{"userId":14,"roomNumber":210}
+]
+);
+
+insert into dataset subscriptions(
+{"param0":123}
+);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f93405cd/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.3.query.aql
new file mode 100644
index 0000000..04b7d28
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.3.query.aql
@@ -0,0 +1,25 @@
+/*
+ * 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 channels;
+
+for $sub in dataset subscriptions
+for $result in currentOccupancy($sub.param0)
+order by $result
+return $result;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f93405cd/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.1.adm b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.1.adm
new file mode 100644
index 0000000..94ebaf9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.1.adm
@@ -0,0 +1,4 @@
+1
+2
+3
+4

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f93405cd/asterix-app/src/test/resources/runtimets/testsuite.xml
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index ac1f8f2..35ecb1e 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -5963,6 +5963,11 @@
                 <expected-error>SyntaxError:  function test.int8@0 is undefined</expected-error>
             </compilation-unit>
         </test-case>
+        <test-case FilePath="user-defined-functions">
+            <compilation-unit name="query-ASTERIXDB-1298">
+                <output-dir compare="Text">query-ASTERIXDB-1298</output-dir>
+            </compilation-unit>
+        </test-case>
         <!-- This test case is not valid anymore since we do not required "IMPORT_PRIVATE_FUNCTIONS" flag anymore -->
         <!-- <test-case FilePath="user-defined-functions">
             <compilation-unit name="invoke-private-function">