You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by wa...@apache.org on 2017/09/28 20:28:11 UTC

asterixdb git commit: [ASTERIXDB-1984][COMP] probe-subtree init not required

Repository: asterixdb
Updated Branches:
  refs/heads/master b67505d6a -> ebde95d63


[ASTERIXDB-1984][COMP] probe-subtree init not required

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Let the IntroduceJoinAccessMethod accept arbitrary
  forms of sub-tree for the probe-tree.

Change-Id: Ib353c85bf627d8dd65dba0ea307dee428edb4a26
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2030
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>


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

Branch: refs/heads/master
Commit: ebde95d63ca626eef2c957e4c025b8f9aa936687
Parents: b67505d
Author: Taewoo Kim <wa...@yahoo.com>
Authored: Thu Sep 28 11:26:07 2017 -0700
Committer: Taewoo Kim <wa...@gmail.com>
Committed: Thu Sep 28 13:27:41 2017 -0700

----------------------------------------------------------------------
 .../rules/am/IntroduceJoinAccessMethodRule.java | 23 ++++---
 .../secondary-equi-join_04.sqlpp                | 44 +++++++++++++
 .../secondary-equi-join_05.sqlpp                | 42 +++++++++++++
 .../secondary-equi-join_06.sqlpp                | 43 +++++++++++++
 .../secondary-equi-join_04.plan                 | 33 ++++++++++
 .../secondary-equi-join_05.plan                 | 18 ++++++
 .../secondary-equi-join_06.plan                 | 38 +++++++++++
 .../btree-index-join/secondary-equi-join_04.ast | 53 ++++++++++++++++
 .../btree-index-join/secondary-equi-join_05.ast | 33 ++++++++++
 .../btree-index-join/secondary-equi-join_06.ast | 66 ++++++++++++++++++++
 10 files changed, 381 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ebde95d6/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index 7fc7902..1a7ccbd 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -320,25 +320,21 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
             // This will be used to find an applicable index on the dataset.
             boolean checkLeftSubTreeMetadata = false;
             boolean checkRightSubTreeMetadata = false;
-            if (continueCheck && (matchInLeftSubTree || matchInRightSubTree)) {
+            if (continueCheck && matchInRightSubTree) {
                 // Set dataset and type metadata.
                 if (matchInLeftSubTree) {
                     checkLeftSubTreeMetadata = leftSubTree.setDatasetAndTypeMetadata(metadataProvider);
                 }
-                if (matchInRightSubTree) {
-                    checkRightSubTreeMetadata = rightSubTree.setDatasetAndTypeMetadata(metadataProvider);
-                }
+                checkRightSubTreeMetadata = rightSubTree.setDatasetAndTypeMetadata(metadataProvider);
             }
 
-            if (continueCheck && (checkLeftSubTreeMetadata || checkRightSubTreeMetadata)) {
+            if (continueCheck && checkRightSubTreeMetadata) {
                 // Map variables to the applicable indexes and find the field name and type.
                 // Then find the applicable indexes for the variables used in the JOIN condition.
                 if (checkLeftSubTreeMetadata) {
                     fillSubTreeIndexExprs(leftSubTree, analyzedAMs, context);
                 }
-                if (checkRightSubTreeMetadata) {
-                    fillSubTreeIndexExprs(rightSubTree, analyzedAMs, context);
-                }
+                fillSubTreeIndexExprs(rightSubTree, analyzedAMs, context);
 
                 // Prune the access methods based on the function expression and access methods.
                 pruneIndexCandidates(analyzedAMs, context, typeEnvironment);
@@ -414,15 +410,18 @@ public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethod
         }
         joinCond = (AbstractFunctionCallExpression) condExpr;
 
-        boolean leftSubTreeInitialized = leftSubTree.initFromSubTree(joinOp.getInputs().get(0));
+        // The result of the left subtree initialization does not need to be checked since only the field type
+        // of the field that is being joined is important. However, if we do not initialize the left sub tree,
+        // we lose a chance to get the field type of a field if there is an enforced index on it.
+        leftSubTree.initFromSubTree(joinOp.getInputs().get(0));
         boolean rightSubTreeInitialized = rightSubTree.initFromSubTree(joinOp.getInputs().get(1));
 
-        if (!leftSubTreeInitialized || !rightSubTreeInitialized) {
+        if (!rightSubTreeInitialized) {
             return false;
         }
 
-        // One of the subtrees must have a datasource scan.
-        if (leftSubTree.hasDataSourceScan() || rightSubTree.hasDataSourceScan()) {
+        // The right (inner) subtree must have a datasource scan.
+        if (rightSubTree.hasDataSourceScan()) {
             return true;
         }
         return false;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ebde95d6/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_04.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_04.sqlpp
new file mode 100644
index 0000000..15107b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_04.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description    : Joins three datasets.
+ *                : Since the given dataset in the inner branch has a secondary index on the field
+ *                : that is being joined, we expect this join to be transformed into an indexed nested-loop join.
+ * Issue          : ASTERIXDB-1984
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestType as
+{
+  id : integer,
+  val : integer
+};
+
+create dataset testdst(TestType) primary key id;
+create dataset testdst2(TestType) primary key id;
+create dataset testdst3(TestType) primary key id;
+
+create index sec3_Idx on testdst3(val) type btree;
+
+SELECT * FROM
+testdst a JOIN testdst2 b ON a.val = b.val JOIN testdst3 c ON b.val /* +indexnl */ =  c.val;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ebde95d6/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_05.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_05.sqlpp
new file mode 100644
index 0000000..31fc953
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_05.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description    : Joins a constant array with a dataset.
+ *                : Since the given dataset in the inner branch has a secondary index on the field that is being joined,
+ *                : we expect this join to be transformed into an indexed nested-loop join.
+ * Issue          : ASTERIXDB-1984
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+use test;
+
+create type TestType as
+{
+  id : integer,
+  val : integer
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (val) type btree;
+
+SELECT * FROM
+[1, 2, 3] AS bar JOIN testdst ON bar /* +indexnl */ = testdst.val;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ebde95d6/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_06.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_06.sqlpp
new file mode 100644
index 0000000..558e172
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_06.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description    : Joins two datasets.
+ *                : Since the given dataset in the inner branch has a secondary index on the field that is being joined,
+ *                : we expect this join to be transformed into an indexed nested-loop join.
+ * Issue          : ASTERIXDB-1984
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+use test;
+
+create type TestType as
+{
+  id : integer,
+  val : integer
+};
+
+create  dataset testdst(TestType) primary key id;
+create  dataset testdst2(TestType) primary key id;
+
+create  index sec_Idx on testdst2(val) type btree;
+
+SELECT * FROM
+(SELECT val, COUNT(*) FROM testdst GROUP BY val) AS bar JOIN testdst2 ON bar.val /* +indexnl */ = testdst2.val;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ebde95d6/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_04.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_04.plan
new file mode 100644
index 0000000..9f60440
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_04.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$19][$$15]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$19]  |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 [$$15]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |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/asterixdb/blob/ebde95d6/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_05.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_05.plan
new file mode 100644
index 0000000..f372412
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_05.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |UNPARTITIONED|
+                                -- UNNEST  |UNPARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ebde95d6/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
new file mode 100644
index 0000000..d7a7847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SORT_GROUP_BY[$$30]  |PARTITIONED|
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                        -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
+                                          -- SORT_GROUP_BY[$$23]  |PARTITIONED|
+                                                  {
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                  }
+                                            -- ONE_TO_ONE_EXCHANGE  |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|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ebde95d6/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_04.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_04.ast
new file mode 100644
index 0000000..fc898df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_04.ast
@@ -0,0 +1,53 @@
+DataverseUse test
+TypeDecl TestType [
+  open RecordType {
+    id : integer,
+    val : integer
+  }
+]
+DatasetDecl testdst(TestType) partitioned by [[id]]
+DatasetDecl testdst2(TestType) partitioned by [[id]]
+DatasetDecl testdst3(TestType) partitioned by [[id]]
+Query:
+SELECT [
+*
+]
+FROM [  FunctionCall Metadata.dataset@1[
+    LiteralExpr [STRING] [testdst]
+  ]
+  AS Variable [ Name=$a ]
+  INNER JOIN
+    FunctionCall Metadata.dataset@1[
+      LiteralExpr [STRING] [testdst2]
+    ]
+    AS Variable [ Name=$b ]
+    ON
+    OperatorExpr [
+      FieldAccessor [
+        Variable [ Name=$a ]
+        Field=val
+      ]
+      =
+      FieldAccessor [
+        Variable [ Name=$b ]
+        Field=val
+      ]
+    ]
+  INNER JOIN
+    FunctionCall Metadata.dataset@1[
+      LiteralExpr [STRING] [testdst3]
+    ]
+    AS Variable [ Name=$c ]
+    ON
+    OperatorExpr [
+      FieldAccessor [
+        Variable [ Name=$b ]
+        Field=val
+      ]
+      =
+      FieldAccessor [
+        Variable [ Name=$c ]
+        Field=val
+      ]
+    ]
+]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ebde95d6/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_05.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_05.ast
new file mode 100644
index 0000000..589b5e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_05.ast
@@ -0,0 +1,33 @@
+DataverseUse test
+TypeDecl TestType [
+  open RecordType {
+    id : integer,
+    val : integer
+  }
+]
+DatasetDecl testdst(TestType) partitioned by [[id]]
+Query:
+SELECT [
+*
+]
+FROM [  OrderedListConstructor [
+    LiteralExpr [LONG] [1]
+    LiteralExpr [LONG] [2]
+    LiteralExpr [LONG] [3]
+  ]
+  AS Variable [ Name=$bar ]
+  INNER JOIN
+    FunctionCall Metadata.dataset@1[
+      LiteralExpr [STRING] [testdst]
+    ]
+    AS Variable [ Name=$testdst ]
+    ON
+    OperatorExpr [
+      Variable [ Name=$bar ]
+      =
+      FieldAccessor [
+        Variable [ Name=$testdst ]
+        Field=val
+      ]
+    ]
+]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ebde95d6/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_06.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_06.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_06.ast
new file mode 100644
index 0000000..141ee40
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_06.ast
@@ -0,0 +1,66 @@
+DataverseUse test
+TypeDecl TestType [
+  open RecordType {
+    id : integer,
+    val : integer
+  }
+]
+DatasetDecl testdst(TestType) partitioned by [[id]]
+DatasetDecl testdst2(TestType) partitioned by [[id]]
+Query:
+SELECT [
+*
+]
+FROM [  (
+    SELECT [
+    Variable [ Name=$val ]
+    val
+    FunctionCall asterix.sql-count@1[
+      (
+        SELECT ELEMENT [
+        LiteralExpr [LONG] [1]
+        ]
+        FROM [          Variable [ Name=#1 ]
+          AS Variable [ Name=#2 ]
+        ]
+      )
+    ]
+    null
+    ]
+    FROM [      FunctionCall Metadata.dataset@1[
+        LiteralExpr [STRING] [testdst]
+      ]
+      AS Variable [ Name=$testdst ]
+    ]
+    Groupby
+      Variable [ Name=$val ]
+      :=
+      FunctionCall Metadata.resolve@-1[
+        LiteralExpr [STRING] [val]
+        Variable [ Name=$testdst ]
+      ]
+      GROUP AS Variable [ Name=#1 ]
+      (
+        testdst:=Variable [ Name=$testdst ]
+      )
+
+  )
+  AS Variable [ Name=$bar ]
+  INNER JOIN
+    FunctionCall Metadata.dataset@1[
+      LiteralExpr [STRING] [testdst2]
+    ]
+    AS Variable [ Name=$testdst2 ]
+    ON
+    OperatorExpr [
+      FieldAccessor [
+        Variable [ Name=$bar ]
+        Field=val
+      ]
+      =
+      FieldAccessor [
+        Variable [ Name=$testdst2 ]
+        Field=val
+      ]
+    ]
+]