You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org> on 2016/08/26 13:56:55 UTC

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin. The...

Wenhai Li has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/1119

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin. The following commits from your working branch will be included:
......................................................................

ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
The following commits from your working branch will be included:

1. Add the test case of ASTERIX-1487 with half records required.

commit 957aaa59f279b53c7901758f88a1820dd8b689c9
Author: Michael <lw...@yahoo.com>
Date:   Fri Aug 26 06:53:00 2016 -0700

    ASTERIX-1487: fix the wrong plan for inverted index.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_4.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
5 files changed, 108 insertions(+), 0 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/1

diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
new file mode 100644
index 0000000..811c2b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
@@ -0,0 +1,45 @@
+/*
+ * 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 fuzzyjoin if exists;
+
+create dataverse fuzzyjoin;
+
+use dataverse fuzzyjoin;
+
+create type DBLPType as open {
+  id: int64,
+  dblpid: string?,
+  title: string?,
+  authors: string?,
+  misc: string?
+}
+
+create type CSXType as open {
+  id: int64,
+  csxid: string?,
+  title: string?,
+  authors: string?,
+  misc: string?
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset CSX(CSXType) primary key id;
+
+create index author_index on DBLP(authors) type keyword;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
new file mode 100644
index 0000000..fc2fb4b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
@@ -0,0 +1,28 @@
+/*
+ * 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 fuzzyjoin;
+
+load dataset DBLP
+using localfs
+(("path"="asterix_nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+load dataset CSX
+using localfs
+(("path"="asterix_nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"),("quote"="\u0000"));
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
new file mode 100644
index 0000000..e9fdc2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
@@ -0,0 +1,29 @@
+/*
+ * 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 fuzzyjointest;
+
+set simthreshold '.7f'
+
+let $s := count(
+for $t in dataset('CSX')
+for $o in dataset('DBLP')
+where contains($o.title, "System") and
+word-tokens($o.authors) ~= word-tokens($t.authors)
+return $t)
+return $s
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_4.1.adm
new file mode 100644
index 0000000..00750ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_4.1.adm
@@ -0,0 +1 @@
+3
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index bf1893a..a526676 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -2651,6 +2651,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="fuzzyjoin">
+      <compilation-unit name="dblp-csx-aqlplus_5">
+        <output-dir compare="Text">dblp-csx-aqlplus_5</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fuzzyjoin">
       <compilation-unit name="dblp-csx-dblp-aqlplus_1">
         <output-dir compare="Text">dblp-csx-dblp-aqlplus_1</output-dir>
       </compilation-unit>

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we prune the selecti...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has submitted this change and it was merged.

Change subject: ASTERIXDB-1487: fix the wrong plan when we prune the selective branch.
......................................................................


ASTERIXDB-1487: fix the wrong plan when we prune the selective branch.

1. Add the test case of ASTERIX-1487 with single join branch required.
2. Disable the join branch pruning in case of unnestmap following datasourcescan.
   - We need to prune the join branch when it is NOT required by the upstream operators and its generated join key is derived from the same DATASOURCE of the other branch.
   - We SHOULD NOT prune the join branch if there exists a selective operator (UNNESTMAP, LOUNNESTMAP, LIMIT, SELECT) located between the join operator and DATASOURCESCAN.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1119
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <bu...@gmail.com>
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/select-self-join.aql
M asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
A asterixdb/asterix-app/src/test/resources/optimizerts/results/select-self-join.plan
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
10 files changed, 225 insertions(+), 21 deletions(-)

Approvals:
  Yingyi Bu: Looks good to me, approved
  Jenkins: Verified; Verified

Objections:
  Jenkins: Violations found



diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
index 2e43912..03c7663 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
@@ -50,6 +50,7 @@
  * 1. The live variables of one input branch of the join are not used in the upstream plan
  * 2. The join is an inner equi join
  * 3. The join condition only uses variables that correspond to primary keys of the same dataset
+ * 4. The records of one input branch will not be filtered by the selective operators till join.
  * Notice that the last condition implies a 1:1 join, i.e., the join does not change the result cardinality.
  * Joins that satisfy the above conditions may be introduced by other rules
  * which use surrogate optimizations. Such an optimization aims to reduce data copies and communication costs by
@@ -61,11 +62,11 @@
  */
 public class RemoveUnusedOneToOneEquiJoinRule implements IAlgebraicRewriteRule {
 
-    private final Set<LogicalVariable> parentsUsedVars = new HashSet<LogicalVariable>();
-    private final List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
-    private final List<LogicalVariable> liveVars = new ArrayList<LogicalVariable>();
-    private final List<LogicalVariable> pkVars = new ArrayList<LogicalVariable>();
-    private final List<DataSourceScanOperator> dataScans = new ArrayList<DataSourceScanOperator>();
+    private final Set<LogicalVariable> parentsUsedVars = new HashSet<>();
+    private final List<LogicalVariable> usedVars = new ArrayList<>();
+    private final List<LogicalVariable> liveVars = new ArrayList<>();
+    private final List<LogicalVariable> pkVars = new ArrayList<>();
+    private final List<DataSourceScanOperator> dataScans = new ArrayList<>();
     private boolean hasRun = false;
 
     @Override
@@ -179,9 +180,35 @@
             // keys from datasource scans of the same dataset.
             return -1;
         }
+        // Suppose we Project B over A.a ~= B.b, where A's fields are involved in a selective operator.
+        // We expect the post-plan will NOT prune the join part derived from A.
+        if (unusedJoinBranchIndex >= 0
+                && isSelectionAboveDataScan(opRef.getValue().getInputs().get(unusedJoinBranchIndex))) {
+            unusedJoinBranchIndex = -1;
+        }
         return unusedJoinBranchIndex;
     }
 
+    private boolean isSelectionAboveDataScan(Mutable<ILogicalOperator> opRef) {
+        boolean hasSelection = false;
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        LogicalOperatorTag tag = op.getOperatorTag();
+        switch (tag) {
+            case DATASOURCESCAN:
+                return false;
+            case UNNEST_MAP:
+            case LEFT_OUTER_UNNEST_MAP:
+            case LIMIT:
+            case SELECT:
+                return true;
+            default:
+                for (Mutable<ILogicalOperator> inputOp : op.getInputs()) {
+                    hasSelection |= isSelectionAboveDataScan(inputOp);
+                }
+        }
+        return hasSelection;
+    }
+
     private void gatherProducingDataScans(Mutable<ILogicalOperator> opRef, List<LogicalVariable> joinUsedVars,
             List<DataSourceScanOperator> dataScans) {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 0bc5e78..8f42904 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -65,6 +65,7 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
@@ -73,7 +74,6 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.LogicalOperatorDeepCopyWithNewVariablesVisitor;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/select-self-join.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/select-self-join.aql
new file mode 100644
index 0000000..720bacd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/select-self-join.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.
+ */
+/*
+ * Description    : Tests that self-join on primary key with select introduces surrogate join.
+ * Success        : Yes
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to asterix_nc1:"opttest/select-self-join.adm";
+
+create type empType as open {
+id: int,
+sal: int
+}
+
+create dataset Emps(empType) primary key id;
+
+for $e1 in dataset Emps
+for $e2 in (for $e3 in dataset Emps where $e3.sal > 1000 return $e3)
+where $e1.id = $e2.id
+return $e1
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
index 4e40dd2..5b08bf5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
@@ -13,20 +13,33 @@
               -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                 -- HASH_PARTITION_EXCHANGE [$$25]  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$36][$$25]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- BTREE_SEARCH  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$25]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
-                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        -- STREAM_SELECT  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/select-self-join.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/select-self-join.plan
new file mode 100644
index 0000000..45c0e3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/select-self-join.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- HYBRID_HASH_JOIN [$$11][$$12]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
new file mode 100644
index 0000000..811c2b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
@@ -0,0 +1,45 @@
+/*
+ * 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 fuzzyjoin if exists;
+
+create dataverse fuzzyjoin;
+
+use dataverse fuzzyjoin;
+
+create type DBLPType as open {
+  id: int64,
+  dblpid: string?,
+  title: string?,
+  authors: string?,
+  misc: string?
+}
+
+create type CSXType as open {
+  id: int64,
+  csxid: string?,
+  title: string?,
+  authors: string?,
+  misc: string?
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset CSX(CSXType) primary key id;
+
+create index author_index on DBLP(authors) type keyword;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
new file mode 100644
index 0000000..fc2fb4b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
@@ -0,0 +1,28 @@
+/*
+ * 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 fuzzyjoin;
+
+load dataset DBLP
+using localfs
+(("path"="asterix_nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+load dataset CSX
+using localfs
+(("path"="asterix_nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"),("quote"="\u0000"));
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
new file mode 100644
index 0000000..7c65b3b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
@@ -0,0 +1,29 @@
+/*
+ * 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 fuzzyjoin;
+
+set simthreshold '.7f'
+
+let $s := count(
+for $t in dataset('CSX')
+for $o in dataset('DBLP')
+where contains($o.title, "System") and
+word-tokens($o.authors) ~= word-tokens($t.authors)
+return $t)
+return $s
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.adm
new file mode 100644
index 0000000..00750ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.adm
@@ -0,0 +1 @@
+3
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index 5750b28..2f277cc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -2656,6 +2656,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="fuzzyjoin">
+      <compilation-unit name="dblp-csx-aqlplus_5">
+        <output-dir compare="Text">dblp-csx-aqlplus_5</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fuzzyjoin">
       <compilation-unit name="dblp-csx-dblp-aqlplus_1">
         <output-dir compare="Text">dblp-csx-dblp-aqlplus_1</output-dir>
       </compilation-unit>

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 16
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1119

to look at the new patch set (#3).

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................

ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

1. Add the test case of ASTERIX-1487 with half records required.

commit 957aaa59f279b53c7901758f88a1820dd8b689c9
Author: Michael <lw...@yahoo.com>
Date:   Fri Aug 26 06:53:00 2016 -0700

    ASTERIX-1487: fix the wrong plan for inverted index.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_4.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
5 files changed, 108 insertions(+), 0 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2414/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we prune the selecti...

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1119

to look at the new patch set (#15).

Change subject: ASTERIXDB-1487: fix the wrong plan when we prune the selective branch.
......................................................................

ASTERIXDB-1487: fix the wrong plan when we prune the selective branch.

1. Add the test case of ASTERIX-1487 with single join branch required.
2. Disable the join branch pruning in case of unnestmap following datasourcescan.
   - We need to prune the join branch when it is NOT required by the upstream operators and its generated join key is derived from the same DATASOURCE of the other branch.
   - We SHOULD NOT prune the join branch if there exists a selective operator (UNNESTMAP, LOUNNESTMAP, LIMIT, SELECT) located between the join operator and DATASOURCESCAN.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/select-self-join.aql
M asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
A asterixdb/asterix-app/src/test/resources/optimizerts/results/select-self-join.plan
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
10 files changed, 225 insertions(+), 21 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/15
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 15
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1119

to look at the new patch set (#4).

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................

ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

1. Add the test case of ASTERIX-1487 with half records required.

commit 957aaa59f279b53c7901758f88a1820dd8b689c9
Author: Michael <lw...@yahoo.com>
Date:   Fri Aug 26 06:53:00 2016 -0700

    ASTERIX-1487: fix the wrong plan for inverted index.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_4.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
5 files changed, 108 insertions(+), 0 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/4
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we prune the selecti...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we prune the selective branch.
......................................................................


Patch Set 14:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/713/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 14
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2412/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we purning the selec...

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Wenhai Li has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we purning the selective branch.
......................................................................


Patch Set 13:

(6 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1119/13//COMMIT_MSG
Commit Message:

Line 7: ASTERIXDB-1487: fix the wrong plan when we purning the selective branch.
> purning -> prune
Done


Line 10: 2. Disable the join branch pruning in case of unnestmap following datasourcescan.
> "unnestmap" -> "operators that do not preserve cardinality"
Yes, my current opinion aims that unnestmap will trigger index-based search, it will "always" introduce selection.


https://asterix-gerrit.ics.uci.edu/#/c/1119/13/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java:

Line 120:     private int removeJoinFromInputBranch(Mutable<ILogicalOperator> opRef, int startBranch) throws AlgebricksException {
> Is variable startBranch necessary?
We have another issue need to be handled.
For example, if we have a join subplan derived from IntroduceAccessMetholdRule especially on a inverted index. The plan will be like:
In short: left branch -> join <- right branch
Or in detail:
datasourcescan(A) -> broadcast -> secondarySearch(B) -> primarySearch(B) -> join <- datasourceScan(A)
Suppose we want to project on A, after applying this patch, we will not prune the left branch. Nevertheless, we should prune the right branch since the required project table can be derived from the left branch, even the referencing variables had been configured towards the right branch by IntroduceIndexAccessMetholdRule. We are trying to handle this case. Once we go ahead with this, the iteration from line 196 will be necessary (Yep, it is not necessary currently!).


Line 137:             if (liveVars.isEmpty()) {
Here, in the context of IntroduceAccessMethodRule, the liveVars derived from the bottom datasourcescan is essential the same yet with a different variable label. We need talk how to ....


Line 196:             return removeJoinFromInputBranch(opRef, 1);
> What's the purpose of the recursive call here?
If the pseudo unusedJoinBranchIndex is excluded by the following SELECTIVE-exception-handler, we need to switch onto the other side if possible.


Line 202:         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
> Call op.getOperatorTag() once after line 202.
Done


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 13
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 11:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2594/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Wenhai Li has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 11:

Unify the two AbstractUnnenstMap exceptions in OneToOnePruning.

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we purning the selec...

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Wenhai Li has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we purning the selective branch.
......................................................................


Patch Set 12:

The condition 3 has been implicitly checked by gatherProducingDataScans, where the dataSourceScanOperator is added onto the list before we verify the join condition contains the PRIMARY KEY of this dataSource. It means that the join condition on the same dataSource will definitely join on the same field (PRIMARY KEY determines anything.).

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 12
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1119

to look at the new patch set (#9).

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................

ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

1. Add the test case of ASTERIX-1487 with half records required.
2. Disable the join branch pruning in case of unnestmap following datasourcescan.

commit 957aaa59f279b53c7901758f88a1820dd8b689c9
Author: Michael <lw...@yahoo.com>
Date:   Fri Aug 26 06:53:00 2016 -0700

    ASTERIXDB-1487: fix the wrong plan for inverted index.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
6 files changed, 141 insertions(+), 17 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/9
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 9
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we purning the selec...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we purning the selective branch.
......................................................................


Patch Set 13:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/695/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 13
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 10: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/601/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 10
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we prune the selecti...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we prune the selective branch.
......................................................................


Patch Set 14: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/713/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 14
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we prune the selecti...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we prune the selective branch.
......................................................................


Patch Set 14:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2750/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 14
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 11:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/1119/11/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java:

Line 159:         for (int i = 0; i < joinOp.getInputs().size(); i++) {
Move the unnest-map gathering/check to line 189. 

Add a method isParentOfUnnestMap(...) to just check if an operator is the parent of an unnestmap.

Call isParentOfUnnestMap(joinOp.getInputs().get(unusedJoinBranchIndex).getValue()) to check the condition at line 189.

One more question:
Should the condition be more general than unnest-map?   Any cardinality reduction operator should be qualified here?   E.g., SELECT etc.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin. The...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin. The following commits from your working branch will be included:
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2405/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2407/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Wenhai Li has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 11:

(5 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1119/11//COMMIT_MSG
Commit Message:

Line 10: 2. Disable the join branch pruning in case of unnestmap following datasourcescan.
> Can you add more explanation on " unnestmap following datasourcescan"?  
Done


Line 11: 
> Remove local commit history in the commit message.
Done


https://asterix-gerrit.ics.uci.edu/#/c/1119/11/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java:

Line 159:         for (int i = 0; i < joinOp.getInputs().size(); i++) {
> Move the unnest-map gathering/check to line 189. 
This is really more general. Let's try it later.


Line 188:         // The pre-plan is (broadcast B and index search on A on each A.a ~= B.b) join B on A.a = B.b
> Can you add the explanation as a comment in the code?  Particularly, explai
Done


https://asterix-gerrit.ics.uci.edu/#/c/1119/11/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
File asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql:

Line 19: 
> Can you a description of this test? What it is intended for? And the expect
Done


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 6: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/475/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1119

to look at the new patch set (#5).

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................

ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

1. Add the test case of ASTERIX-1487 with half records required.
2. Disable the join branch pruning in case of unnestmap following datasourcescan.

commit 957aaa59f279b53c7901758f88a1820dd8b689c9
Author: Michael <lw...@yahoo.com>
Date:   Fri Aug 26 06:53:00 2016 -0700

    ASTERIX-1487: fix the wrong plan for inverted index.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_4.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
6 files changed, 134 insertions(+), 17 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/5
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we purning the selec...

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1119

to look at the new patch set (#12).

Change subject: ASTERIXDB-1487: fix the wrong plan when we purning the selective branch.
......................................................................

ASTERIXDB-1487: fix the wrong plan when we purning the selective branch.

1. Add the test case of ASTERIX-1487 with half records required.
2. Disable the join branch pruning in case of unnestmap following datasourcescan.
   - We need to prune the join branch when it is NOT required by the upstream operators and its generated join key is derived from the same DATASOURCE of the other branch.
   - We SHOULD NOT prune the join branch if there exists a selective operator (UNNESTMAP, LOUNNESTMAP, LIMIT, SELECT) located between the join operator and DATASOURCESCAN.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
7 files changed, 154 insertions(+), 14 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/12
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 12
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we prune the selecti...

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1119

to look at the new patch set (#14).

Change subject: ASTERIXDB-1487: fix the wrong plan when we prune the selective branch.
......................................................................

ASTERIXDB-1487: fix the wrong plan when we prune the selective branch.

1. Add the test case of ASTERIX-1487 with single join branch required.
2. Disable the join branch pruning in case of unnestmap following datasourcescan.
   - We need to prune the join branch when it is NOT required by the upstream operators and its generated join key is derived from the same DATASOURCE of the other branch.
   - We SHOULD NOT prune the join branch if there exists a selective operator (UNNESTMAP, LOUNNESTMAP, LIMIT, SELECT) located between the join operator and DATASOURCESCAN.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/select-self-join.aql
M asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
A asterixdb/asterix-app/src/test/resources/optimizerts/results/select-self-join.plan
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
10 files changed, 227 insertions(+), 22 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/14
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 14
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Wenhai Li has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 11:

(4 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1119/11/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java:

Line 188:         // The pre-plan is (broadcast B and index search on A on each A.a ~= B.b) join B on A.a = B.b
> I couldn't figure out what this comment mean too. Can you rephrase it if po
for $d in dataset A
for $t in dataset B (inverted index on B.b)
where contains($B, "keywords") and word-tokens($d.a) ~= word-tokens($t.b)
return $A
Here, all the "contains" will be pruned even we have a selection condition in this branch, where we should select on the inverted index and then join with the broadcasted A's records. Note here that the project "return $A" only covers the fields of A.


https://asterix-gerrit.ics.uci.edu/#/c/1119/11/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
File asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql:

Line 24: for $t in dataset('CSX')
The second branch will be pruned in the current master.


Line 26: where contains($o.title, "System") and
This condition will be ignored after we applying RemoveUnusedOneToOneEquiJoinRule, which results in the inconsistency when we switch the two branch "for $t ... " and "for $o ...".


Line 28: return $t)
Since we return only $t.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 9:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2573/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 9
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 6:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/475/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2505/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we prune the selecti...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we prune the selective branch.
......................................................................


Patch Set 15: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 15
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Chen Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Chen Li has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 7:

Wenhai: I am trying to review this patch.  To help me (or other reviewers), can you add a high-level description of the cause of the problem, and how you fix it?  In addition, for those major changes (mainly the java file), add a few comments to explain those changes.  That will help a lot!

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Wenhai Li has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 11:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1119/11/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java:

Line 187:         // Suppose we Project B over A.a ~= B.b and, where A.a involves a secondary word index.
> What does Project B mean? Project entire record? or just one or any fields 
Here, B means several selected fields of Table B. Notice here that the exception will occur only when Project includes B's fields, not including A's fields.


Line 192:         }
> Can you explain this part?
Nothing special but to verify whether one branch from 0 or 1 is attached with an AbstractUnnestMap above DataSourceScan, we should keep this selection even no fields will be projected by the top-level operators.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin. The...

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Wenhai Li has uploaded a new patch set (#2).

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin. The following commits from your working branch will be included:
......................................................................

ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
The following commits from your working branch will be included:

1. Add the test case of ASTERIX-1487 with half records required.

commit 957aaa59f279b53c7901758f88a1820dd8b689c9
Author: Michael <lw...@yahoo.com>
Date:   Fri Aug 26 06:53:00 2016 -0700

    ASTERIX-1487: fix the wrong plan for inverted index.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_4.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
5 files changed, 108 insertions(+), 0 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we prune the selecti...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we prune the selective branch.
......................................................................


Patch Set 15:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/716/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 15
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we purning the selec...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we purning the selective branch.
......................................................................


Patch Set 13:

(5 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1119/13//COMMIT_MSG
Commit Message:

Line 7: ASTERIXDB-1487: fix the wrong plan when we purning the selective branch.
purning -> prune


Line 10: 2. Disable the join branch pruning in case of unnestmap following datasourcescan.
"unnestmap" -> "operators that do not preserve cardinality"


https://asterix-gerrit.ics.uci.edu/#/c/1119/13/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java:

Line 120:     private int removeJoinFromInputBranch(Mutable<ILogicalOperator> opRef, int startBranch) throws AlgebricksException {
Is variable startBranch necessary?
Why it didn't work without this parameter?


Line 196:             return removeJoinFromInputBranch(opRef, 1);
What's the purpose of the recursive call here?


Line 202:         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
Call op.getOperatorTag() once after line 202.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 13
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 8: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/548/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1119

to look at the new patch set (#7).

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................

ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

1. Add the test case of ASTERIX-1487 with half records required.
2. Disable the join branch pruning in case of unnestmap following datasourcescan.

commit 957aaa59f279b53c7901758f88a1820dd8b689c9
Author: Michael <lw...@yahoo.com>
Date:   Fri Aug 26 06:53:00 2016 -0700

    ASTERIX-1487: fix the wrong plan for inverted index.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
6 files changed, 135 insertions(+), 17 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/7
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we prune the selecti...

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Wenhai Li has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we prune the selective branch.
......................................................................


Patch Set 16:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/1119/16/asterixdb/asterix-app/src/test/resources/optimizerts/queries/select-self-join.aql
File asterixdb/asterix-app/src/test/resources/optimizerts/queries/select-self-join.aql:

Line 23:  
Sorry for the white space.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 16
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we purning the selec...

Posted by "Taewoo Kim (Code Review)" <do...@asterixdb.incubator.apache.org>.
Taewoo Kim has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we purning the selective branch.
......................................................................


Patch Set 12:

I discussed this with Wenhai. 

For a join such as A.x = B.y, we need to make sure that
1) The datasourcescan of A and B refers to the same dataset (A,B = the same dataset)
2) Operators after the join do not use any variables from one branch.
3) The fields X and Y should be the same field

I am not sure whether the current rule checks the condition 3) properly.

Wenhai also mentioned two issues:
#a. We have a SELECT condition besides to the above (e.g., A.x = B.y and A.z < 100). In this case, We have a datasource scan, followed by a SELECT, then the join op. We can't prune the A branch since if we do so, A.z can't be checked. A solution could be moving A.z < 100 after the join operator and change it to B.z < 100. 

#b. For surrogate join case that utilizes a secondary inverted index-search, this rule always prune the inner branch (which does main job). How to let this recognize the inner branch is an issue.

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 12
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 11:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/612/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Wenhai Li has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 7:

@Chen
Ok, I will give an example with comments ASAP.

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 11:

(3 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1119/11//COMMIT_MSG
Commit Message:

Line 10: 2. Disable the join branch pruning in case of unnestmap following datasourcescan.
Can you add more explanation on " unnestmap following datasourcescan"?  

What does it mean?  An index join?  Why it needs to disable the join branch pruning?


Line 11: 
Remove local commit history in the commit message.


https://asterix-gerrit.ics.uci.edu/#/c/1119/11/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java:

Line 188:         // The pre-plan is (broadcast B and index search on A on each A.a ~= B.b) join B on A.a = B.b
> for $d in dataset A
Can you add the explanation as a comment in the code?  Particularly, explain why the addition if-branch here is necessary and what happens if the if-branch is not added.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 11: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/612/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Taewoo Kim (Code Review)" <do...@asterixdb.incubator.apache.org>.
Taewoo Kim has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 8:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/1119/8//COMMIT_MSG
Commit Message:

Line 18: Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Please check the other commit message and please follow the format.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Taewoo Kim (Code Review)" <do...@asterixdb.incubator.apache.org>.
Taewoo Kim has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 8:

(3 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1119/8/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java:

Line 197:     private void gatherProducingDataScans(Mutable<ILogicalOperator> opRef, Set<UnnestMapOperator> parentWithUM) {
AbstractUnnestMapOperator


Line 206:         } else if (op.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
We also have LEFT_OUTER_UNNEST_MAP. You need to add this too.


Line 208:             parentWithUM.add((UnnestMapOperator) op);
AbstractUnnestMapOperator


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Wenhai Li has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 8:

@Chen
It's a quite strange query, the high-level comments towards the example is like:
1. We setup a fuzzy join over A.a ~= B.b based on a word index over A.a
2. We want to aggregate on B's fields.
If we switch the consequence of A and B in the two "FOR" lines, we get inconsistent result. The RemoveUnusedOneToOneEuquiJoinRule remove the branch of A after "pseudo" broadcase B to A for enabling "SELECT" instead of fuzzy join. But that is wrong.

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we purning the selec...

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1119

to look at the new patch set (#13).

Change subject: ASTERIXDB-1487: fix the wrong plan when we purning the selective branch.
......................................................................

ASTERIXDB-1487: fix the wrong plan when we purning the selective branch.

1. Add the test case of ASTERIX-1487 with half records required.
2. Disable the join branch pruning in case of unnestmap following datasourcescan.
   - We need to prune the join branch when it is NOT required by the upstream operators and its generated join key is derived from the same DATASOURCE of the other branch.
   - We SHOULD NOT prune the join branch if there exists a selective operator (UNNESTMAP, LOUNNESTMAP, LIMIT, SELECT) located between the join operator and DATASOURCESCAN.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
M asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
8 files changed, 182 insertions(+), 28 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/13
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 13
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 10:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2574/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 10
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 11:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/1119/11/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java:

Line 188:         // The pre-plan is (broadcast B and index search on A on each A.a ~= B.b) join B on A.a = B.b
I couldn't figure out what this comment mean too. Can you rephrase it if possible?


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we purning the selec...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we purning the selective branch.
......................................................................


Patch Set 13:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2723/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 13
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we prune the selecti...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we prune the selective branch.
......................................................................


Patch Set 15:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2754/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 15
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we prune the selecti...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we prune the selective branch.
......................................................................


Patch Set 15: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/716/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 15
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 8:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/548/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Chen Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Chen Li has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 8:

@wenhai: Similar to another "fuzzy issue", I talked to @taewoo about this patch.  I suggest you two schedule a skype time to go through the changes together to explain/approve those changes.  Also I think it's critical to make sure proper comments are added, and those comments are clear.  Try to avoid grammatical mistakes in the comments.  Thanks!

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1119

to look at the new patch set (#8).

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................

ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

1. Add the test case of ASTERIX-1487 with half records required.
2. Disable the join branch pruning in case of unnestmap following datasourcescan.

commit 957aaa59f279b53c7901758f88a1820dd8b689c9
Author: Michael <lw...@yahoo.com>
Date:   Fri Aug 26 06:53:00 2016 -0700

    ASTERIX-1487: fix the wrong plan for inverted index.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
6 files changed, 139 insertions(+), 17 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/8
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1119

to look at the new patch set (#10).

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................

ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

- Add the test case of ASTERIXDB-1487 with half records required.
- Disable the join branch pruning in case of AbstractUnnestMap following DataSourceScan.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
6 files changed, 141 insertions(+), 17 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/10
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 10
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1119

to look at the new patch set (#11).

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................

ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

1. Add the test case of ASTERIX-1487 with half records required.
2. Disable the join branch pruning in case of unnestmap following datasourcescan.

commit 957aaa59f279b53c7901758f88a1820dd8b689c9
Author: Michael <lw...@yahoo.com>
Date:   Fri Aug 26 06:53:00 2016 -0700

    ASTERIXDB-1487: fix the wrong plan for inverted index.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
6 files changed, 141 insertions(+), 17 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/11
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2406/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we purning the selec...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we purning the selective branch.
......................................................................


Patch Set 13: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/695/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 13
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan when we purning the selec...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan when we purning the selective branch.
......................................................................


Patch Set 12:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2708/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 12
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 10:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/601/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 10
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Taewoo Kim (Code Review)" <do...@asterixdb.incubator.apache.org>.
Taewoo Kim has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 10:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/1119/10/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java:

Line 208:                 || op.getOperatorTag().equals(LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP)) {
Two if conditions can be the same check - either equals or ==. Please unify them.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 10
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Taewoo Kim (Code Review)" <do...@asterixdb.incubator.apache.org>.
Taewoo Kim has posted comments on this change.

Change subject: ASTERIXDB-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 11:

(3 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1119/11/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java:

Line 187:         // Suppose we Project B over A.a ~= B.b and, where A.a involves a secondary word index.
What does Project B mean? Project entire record? or just one or any fields is OK?


Line 192:         }
Can you explain this part?


https://asterix-gerrit.ics.uci.edu/#/c/1119/11/asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
File asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql:

Line 19: 
Can you a description of this test? What it is intended for? And the expected result?

An example: /apache-asterixdb/asterix-app/src/test/resources/runtimets/queries/index-join/btree-primary-equi-join/btree-primary-equi-join.1.ddl.aql


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 11
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1119

to look at the new patch set (#6).

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................

ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

1. Add the test case of ASTERIX-1487 with half records required.
2. Disable the join branch pruning in case of unnestmap following datasourcescan.

commit 957aaa59f279b53c7901758f88a1820dd8b689c9
Author: Michael <lw...@yahoo.com>
Date:   Fri Aug 26 06:53:00 2016 -0700

    ASTERIX-1487: fix the wrong plan for inverted index.

Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.2.update.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.3.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_5/dblp-csx-aqlplus_5.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
6 files changed, 134 insertions(+), 17 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/1119/6
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 7: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/477/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIX-1487: fix the wrong plan for inverted fuzzyjoin.
......................................................................


Patch Set 7:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/477/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1119
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1aef69a2278853fd9f8020da6639331b367ed5ad
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Chen Li <ch...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@yahoo.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <yi...@google.com>
Gerrit-HasComments: No