[ASTERIXDB-3144][COMP] Change delivered properties of datasets
- user model changes: no
- storage format changes: no
- interface changes: yes
Details:
This patch is to change the delivered properties of a dataset
to allow the separation of compute partitions from the storage
partitions.
Change-Id: Ifcafc443406980f3618e0c041260889816c73d8d
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/17434
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Murtadha Hubail <mhubail@apache.org>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
index fc899c6..9917589 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
@@ -112,7 +112,7 @@
dataset.getDatasetDetails(), context.getComputationNodeDomain());
IDataSourcePropertiesProvider dspp = ds.getPropertiesProvider();
AbstractScanOperator as = (AbstractScanOperator) op;
- deliveredProperties = dspp.computePropertiesVector(as.getVariables());
+ deliveredProperties = dspp.computeDeliveredProperties(as.getVariables());
}
@Override
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
index 4de30ce..d688ab8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
@@ -69,7 +69,7 @@
IDataSource<?> ds = idx.getDataSource();
IDataSourcePropertiesProvider dspp = ds.getPropertiesProvider();
AbstractScanOperator as = (AbstractScanOperator) op;
- deliveredProperties = dspp.computePropertiesVector(as.getScanVariables());
+ deliveredProperties = dspp.computeDeliveredProperties(as.getScanVariables());
}
protected int[] getKeyIndexes(List<LogicalVariable> keyVarList, IOperatorSchema[] inputSchemas) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
index ee018f9..047a6d5 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
@@ -42,6 +42,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
import org.apache.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
@@ -111,15 +112,23 @@
@Override
public IDataSourcePropertiesProvider getPropertiesProvider() {
- return scanVariables -> {
- List<ILocalStructuralProperty> propsLocal = new ArrayList<>(1);
- //TODO(ali): consider primary keys?
- List<OrderColumn> secKeys = new ArrayList<>(numSecKeys);
- for (int i = 0; i < numSecKeys; i++) {
- secKeys.add(new OrderColumn(scanVariables.get(i), OrderOperator.IOrder.OrderKind.ASC));
+ return new IDataSourcePropertiesProvider() {
+ @Override
+ public IPhysicalPropertiesVector computeRequiredProperties(List<LogicalVariable> scanVariables) {
+ return StructuralPropertiesVector.EMPTY_PROPERTIES_VECTOR;
}
- propsLocal.add(new LocalOrderProperty(secKeys));
- return new StructuralPropertiesVector(new RandomPartitioningProperty(domain), propsLocal);
+
+ @Override
+ public IPhysicalPropertiesVector computeDeliveredProperties(List<LogicalVariable> scanVariables) {
+ List<ILocalStructuralProperty> propsLocal = new ArrayList<>(1);
+ //TODO(ali): consider primary keys?
+ List<OrderColumn> secKeys = new ArrayList<>(numSecKeys);
+ for (int i = 0; i < numSecKeys; i++) {
+ secKeys.add(new OrderColumn(scanVariables.get(i), OrderOperator.IOrder.OrderKind.ASC));
+ }
+ propsLocal.add(new LocalOrderProperty(secKeys));
+ return new StructuralPropertiesVector(new RandomPartitioningProperty(domain), propsLocal);
+ }
};
}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
index b596e4b..80caf74 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
@@ -173,7 +173,7 @@
-- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelBrokerSubscriptions) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$239, $$240] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-01.plan
index a3d4489..17b0d76 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-01.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$59(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$61(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-02.plan
index 09d5382..f926a1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-02.plan
@@ -2,18 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$59(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds2.ds2) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$61(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$72(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds2.ds2) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$74(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds2.idx2) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds2.idx2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-03.plan
index 40e63cb..92b3b68 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-03.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$59(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds3) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$61(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds3) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-04.plan
index 3582cf3..88dc9b3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-04.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds3) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$22(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$22(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- DATASOURCE_SCAN (test.ds3) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-10.plan
index 9e39687..16f44e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-10.plan
@@ -2,18 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds5.idx5_dt_fmt) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.idx5_dt_fmt) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-11.plan
index 505a834..297794d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-11.plan
@@ -2,18 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds5.idx5_d_fmt) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.idx5_d_fmt) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-12.plan
index 11a2618..910a579 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-12.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-12.plan
@@ -2,18 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds5.idx5_t_fmt) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.idx5_t_fmt) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-13.plan
index 7736d99..3cc15fb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-13.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-13.plan
@@ -2,18 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds6.idx6_dt) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.idx6_dt) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-14.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-14.plan
index d9444e7..fdd7255 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-14.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-14.plan
@@ -2,18 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds6.idx6_d) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.idx6_d) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-15.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-15.plan
index e4b5cd7..3b5b684 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-15.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-15.plan
@@ -2,18 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds6.idx6_t) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.idx6_t) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-16.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-16.plan
index 753d23d..62d8cf1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-16.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-16.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-17.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-17.plan
index f6647d4..607e59c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-17.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-17.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-18.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-18.plan
index 753d23d..62d8cf1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-18.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-18.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-20.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-20.plan
index d71b03a..62dc0e9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-20.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-20.plan
@@ -2,18 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$60(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$60(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$70(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$72(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds7.idx2) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.idx2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-21.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-21.plan
index e5fd2d4..37bd6b8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-21.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-21.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$60(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$60(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-22.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-22.plan
index e5fd2d4..37bd6b8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-22.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-22.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$60(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$60(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-23.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-23.plan
index 5972aae..9d2902b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-23.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-23.plan
@@ -2,18 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$60(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$60(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$70(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$72(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds7.idx3) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.idx3) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-24.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-24.plan
index 85d3144..a83a1a9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-24.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-24.plan
@@ -2,18 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds7.idx1) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- BTREE_SEARCH (test.ds7.idx1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-25.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-25.plan
index d71b03a..62dc0e9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-25.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-25.plan
@@ -2,18 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$60(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$60(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$70(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$72(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.ds7.idx2) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.idx2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/disjunctive-predicate/disjunctive-predicate-5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/disjunctive-predicate/disjunctive-predicate-5.plan
index 242b032..7f72a7e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/disjunctive-predicate/disjunctive-predicate-5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/disjunctive-predicate/disjunctive-predicate-5.plan
@@ -1,11 +1,11 @@
-- DISTRIBUTE_RESULT |UNPARTITIONED|
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
-- STREAM_LIMIT |UNPARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- -- STREAM_LIMIT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- STREAM_LIMIT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.TestSet) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01-disable-idxonly.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01-disable-idxonly.plan
index aa1f23c..f37ce33 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01-disable-idxonly.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01-disable-idxonly.plan
@@ -2,18 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.MyData.MyData) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.MyData.MyData) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.MyData.btree_index_docid) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (test.MyData.btree_index_docid) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan
index 02a8214..6c4b299 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan
@@ -4,15 +4,15 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$30][$$31] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$32][$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (colocated.Users) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan
index 616e808..e68bf78 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan
@@ -3,13 +3,13 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$185, $$186] |PARTITIONED|
+ -- SORT_GROUP_BY[$$194, $$195] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$185, $$186] |PARTITIONED|
- -- SORT_GROUP_BY[$$168, $$169] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$194, $$195] |PARTITIONED|
+ -- SORT_GROUP_BY[$$177, $$178] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
@@ -26,15 +26,15 @@
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$172] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$181] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$172(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$181(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$181] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- NESTED_LOOP |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/composite-key/composite-prefix-low-high.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/composite-key/composite-prefix-low-high.plan
index 5639621..4d5fe26 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/composite-key/composite-prefix-low-high.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/composite-key/composite-prefix-low-high.plan
@@ -2,8 +2,10 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$25(ASC), $$26(ASC) ] |PARTITIONED|
- -- BTREE_SEARCH (test.Points.Points) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$26(ASC), $$27(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$26(ASC), $$27(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (test.Points.Points) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
index 5516a6f..f2f336d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
@@ -3,22 +3,22 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$93] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$99] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$93(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$99(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$93][$$95] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$99][$$101] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$99] |PARTITIONED|
-- DATASOURCE_SCAN (custorder.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$95] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$101] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan
index b6f3cb4..51da71e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan
@@ -8,11 +8,11 @@
-- INSERT_DELETE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- MATERIALIZE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$19] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$19][$$21] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$20][$$22] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_4.plan
index 00c7dfd..fd34038 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_4.plan
@@ -28,7 +28,7 @@
-- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
-- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_5.plan
index b5ed081..c1e55d8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_5.plan
@@ -20,7 +20,7 @@
-- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
-- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_6.plan
index 00c7dfd..fd34038 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_6.plan
@@ -28,7 +28,7 @@
-- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
-- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_7.plan
index 00c7dfd..fd34038 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_7.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/broadcast_join_hint/broadcast_join_hint_7.plan
@@ -28,7 +28,7 @@
-- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
-- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_1.plan
index c5b8dc2..8f0ab1f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_1.plan
@@ -20,7 +20,7 @@
-- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
-- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_2.plan
index 5610321..3e1214a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_2.plan
@@ -28,7 +28,7 @@
-- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
-- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_3.plan
index 5610321..3e1214a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_3.plan
@@ -28,7 +28,7 @@
-- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
-- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_4.plan
index 5610321..3e1214a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_4.plan
@@ -28,7 +28,7 @@
-- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
-- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_5.plan
index c5b8dc2..8f0ab1f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_5.plan
@@ -20,7 +20,7 @@
-- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
-- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_6.plan
index c5b8dc2..8f0ab1f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_6.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/hashjoin_hint/hashjoin_hint_6.plan
@@ -20,7 +20,7 @@
-- DATASOURCE_SCAN (tpch.customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
-- DATASOURCE_SCAN (tpch.nation) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
index d122327..bff246a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
@@ -1,11 +1,11 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$42(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$45(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$43][$$44] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$46][$$47] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -13,10 +13,12 @@
-- DATASOURCE_SCAN (fuzzyjoin.TOKENSRANKEDADM) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$42(ASC)] HASH:[$$44] |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$45(ASC)] HASH:[$$47] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (fuzzyjoin.TOKENSRANKEDADM) |PARTITIONED|
+ -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (fuzzyjoin.TOKENSRANKEDADM) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan
index 8a9e3fa..da22292 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan
@@ -1,9 +1,11 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$15(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$16(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
index 540bb89..005ab46 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
@@ -1,14 +1,16 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$15(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$16(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index) |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan
index 7434db4..d478d3e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan
@@ -1,9 +1,11 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$17(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$18(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
index 719fa63..971ae76 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
@@ -1,14 +1,16 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$17(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$18(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
+ -- STABLE_SORT [$$22(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan
index 7434db4..d478d3e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan
@@ -1,9 +1,11 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$17(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$18(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
index 719fa63..971ae76 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
@@ -1,14 +1,16 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$17(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$18(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
+ -- STABLE_SORT [$$22(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
index a4d6af9..ad32edd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
@@ -1,14 +1,16 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$16(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$17(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan
index 8a9e3fa..da22292 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan
@@ -1,9 +1,11 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$15(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$16(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan
index 251929c..801b97f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan
@@ -1,9 +1,11 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$27(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
index 4ab6ffa..5b54be6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
@@ -1,14 +1,16 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$27(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
+ -- STABLE_SORT [$$33(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
index bbc5404..f4bdc09 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
@@ -4,15 +4,15 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$58][$$49] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$61][$$52] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$61] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$52] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -22,7 +22,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$64(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
index df95f20..6549e08 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
@@ -4,15 +4,15 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$58][$$49] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$61][$$52] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$61] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$52] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -22,7 +22,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$64(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
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 38e2902..075b602 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
@@ -3,19 +3,19 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$57] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$60] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$57(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$57] |PARTITIONED|
+ -- STABLE_SORT [$$60(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$60] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$68][$$57] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$71][$$60] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -24,14 +24,14 @@
-- DATASOURCE_SCAN (test.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$57] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$60] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$71(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$74(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicIIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan
index a49ca8f..98a544e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan
@@ -2,20 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$69(ASC) ] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$59] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$72(ASC) ] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$62] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC), $$60(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC), $$63(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$75][$$59] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$78][$$62] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$78] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -25,7 +25,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -36,7 +36,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$83(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
index acca388..86e90b4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
@@ -3,25 +3,25 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$69(ASC)] |PARTITIONED|
- -- RANGE_PARTITION_EXCHANGE [$$69(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$72(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$72(ASC)] |PARTITIONED|
-- FORWARD |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$59] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$62] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC), $$60(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC), $$63(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$75][$$59] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$78][$$62] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$78] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -31,7 +31,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -42,7 +42,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$83(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
@@ -90,19 +90,19 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$59] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$62] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC), $$60(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC), $$63(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$75][$$59] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$78][$$62] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$78] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -112,7 +112,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -123,7 +123,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$83(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
index a0c5597..34268a6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
@@ -2,20 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$69(ASC) ] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$59] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$72(ASC) ] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$62] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC), $$60(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC), $$63(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$75][$$59] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$78][$$62] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$78] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -25,7 +25,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -33,7 +33,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$79(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$82(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
index 0151e1f..0b3fbc2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
@@ -3,25 +3,25 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$69(ASC)] |PARTITIONED|
- -- RANGE_PARTITION_EXCHANGE [$$69(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$72(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$72(ASC)] |PARTITIONED|
-- FORWARD |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$59] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$62] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC), $$60(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC), $$63(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$75][$$59] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$78][$$62] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$78] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -31,7 +31,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -39,7 +39,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$79(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$82(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
@@ -60,19 +60,19 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$59] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$62] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC), $$60(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC), $$63(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$75][$$59] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$78][$$62] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$78] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -82,7 +82,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -90,7 +90,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$79(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$82(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
index 84bdda2..801cb44 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.CSX.CSX) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
index eb0c5cb..05a0087 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
index 3ebf17f..4231828 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$54][$$44] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$57][$$47] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$57] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$44] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -20,7 +20,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
index 84bdda2..801cb44 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.CSX.CSX) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
index eb0c5cb..05a0087 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
index 21ec002..e549036 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$54][$$43] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$57][$$46] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$57] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -20,7 +20,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
index fd02f50..9b07d54 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$40][$$32] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$42][$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$42] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.CSX.CSX) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$47(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
index 0d31425..a63ddbb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$40][$$32] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$42][$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$42] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$47(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
index 4231897..d9a606c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$44][$$34] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$46][$$36] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$36] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.CSX.CSX) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$47(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$49(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
index 6993ea0..58573b2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$44][$$34] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$46][$$36] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$36] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$47(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$49(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
index 25dffe2..a12dce8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$45][$$35] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$47][$$37] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.CSX.CSX) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
index 760eb13..8281cb4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$45][$$35] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$47][$$37] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
index 69b1d36..5502ec5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$58][$$46] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$61][$$49] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$61] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$64(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
index 25dffe2..a12dce8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$45][$$35] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$47][$$37] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.CSX.CSX) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
index 760eb13..8281cb4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$45][$$35] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$47][$$37] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
index a037692..a05aede 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$58][$$45] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$61][$$48] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$61] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$45] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$48] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$64(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.ngram_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
index be6eb79..798833d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers2.Customers2) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
index cd50a99..855017c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
index 05a02d8..50468e9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$54][$$44] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$57][$$47] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$57] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$44] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -20,7 +20,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
index be6eb79..798833d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers2.Customers2) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
index cd50a99..855017c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
index a5ea5f3..4d65bc4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$54][$$43] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$57][$$46] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$57] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -20,7 +20,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
index 2797eef..f7a5011 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$40][$$32] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$42][$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$42] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers2.Customers2) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$47(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
index 9d950ce..b8e3d79 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$40][$$32] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$42][$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$42] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$47(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
index 8e921c5..c13d83b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$40][$$32] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$42][$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$42] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers2.Customers2) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
index fe7125d..1e7b2ca 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$40][$$32] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$42][$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$42] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
index 8350b49..41af3ae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers2.Customers2) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
index 2682723..6dfa31d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
index c482198..8c62a6d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$54][$$44] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$57][$$47] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$57] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$44] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
index 8350b49..41af3ae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers2.Customers2) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
index 2682723..6dfa31d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
index 173a0b2..782070e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$54][$$43] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$57][$$46] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$57] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
index 8e921c5..c13d83b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$40][$$32] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$42][$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$42] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers2.Customers2) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
index fe7125d..1e7b2ca 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$40][$$32] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$42][$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$42] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
index 8350b49..41af3ae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers2.Customers2) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
index 2682723..6dfa31d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
index c482198..8c62a6d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$54][$$44] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$57][$$47] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$57] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$44] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
index 8350b49..41af3ae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers2.Customers2) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers2.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
index 2682723..6dfa31d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$33] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
index 173a0b2..782070e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$54][$$43] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$57][$$46] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$57] |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.Customers.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.Customers.interests_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
index c0704d0..b9091a3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$44][$$34] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$46][$$36] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$36] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.CSX.CSX) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$47(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$49(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.keyword_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
index 002d0c5..f364b67 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$44][$$34] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$46][$$36] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$36] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$47(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$49(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
index 0767902..744eb2f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
@@ -4,8 +4,8 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$66][$$53] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$69][$$56] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$69] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$56] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -26,7 +26,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$71(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$74(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgTextIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
index 40582ac..2663394 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$45][$$35] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$47][$$37] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.CSX.CSX) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.keyword_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
index c752bfe..c7b7f6d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$45][$$35] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$47][$$37] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
index ee21d84..a7f501c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$58][$$46] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$61][$$49] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$61] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$64(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
index 40582ac..2663394 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$45][$$35] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$47][$$37] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.CSX.CSX) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.CSX.keyword_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
index c752bfe..c7b7f6d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$45][$$35] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$47][$$37] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
index 627e111..bfea2fb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$58][$$45] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$61][$$48] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$61] |PARTITIONED|
-- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$45] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$48] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DBLP.DBLP) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$64(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.DBLP.keyword_index) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
index 2526adf..6bae2cc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
@@ -4,8 +4,8 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$37, $$42, $$44][$$39, $$43, $$40] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$37, $$44] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$39, $$44, $$46][$$41, $$45, $$42] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$39, $$44, $$46] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -13,7 +13,7 @@
-- DATASOURCE_SCAN (`join-super-key_1`.LineItems) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$41, $$45, $$42] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
index 5010ca8..da9d41a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
@@ -4,15 +4,15 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$37, $$43, $$38][$$39, $$42, $$44] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$39, $$45, $$40][$$41, $$44, $$46] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$39, $$45, $$40] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (`join-super-key_01`.PartSupp) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$39, $$44] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$41, $$44, $$46] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/inner_right_corr.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/inner_right_corr.plan
index 7a7a543..8327ffc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/inner_right_corr.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/inner_right_corr.plan
@@ -4,7 +4,7 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$71] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$76] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- AGGREGATE |LOCAL|
@@ -12,42 +12,44 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$76(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$71][$$87] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.t1) |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$76][$$92] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$76] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- UNNEST |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- SUBPLAN |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- ASSIGN |LOCAL|
- -- UNNEST |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
+ -- DATASOURCE_SCAN (test.t1) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- NESTED_LOOP |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$92] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.t1) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.t2) |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.t1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.t2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan
index 15c163d..a9d6547 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan
@@ -2,7 +2,7 @@
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -10,18 +10,18 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$38] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$40] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$38(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$40(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$40] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$41][$$39] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$43][$$41] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/query-ASTERIXDB-2857.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/query-ASTERIXDB-2857.plan
index 393f1db..18934ba 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/query-ASTERIXDB-2857.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/query-ASTERIXDB-2857.plan
@@ -2,8 +2,8 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$136(ASC), $$137(ASC), $#3(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$136(ASC), $$137(ASC), $#3(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$145(ASC), $$146(ASC), $#3(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$145(ASC), $$146(ASC), $#3(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -12,8 +12,8 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$127][$$128] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$136][$$137] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$136] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -21,7 +21,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
index ad0ffd7..5d1ba2b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
@@ -4,25 +4,25 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$46, $$47] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$49, $$50] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$46(ASC), $$47(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$46, $$47] |PARTITIONED|
+ -- STABLE_SORT [$$49(ASC), $$50(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$49, $$50] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$50, $$46, $$54][$$51, $$48, $$49] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$46, $$54] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$53, $$49, $$57][$$54, $$51, $$52] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$53, $$49, $$57] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (`loj-super-key_01`.LineItems) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$54, $$51, $$52] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (`loj-super-key_01`.PartSupp) |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
index b2ca8e1..ccad49d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
@@ -4,25 +4,25 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$46, $$47] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$49, $$50] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$46(ASC), $$47(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$49(ASC), $$50(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$49, $$50] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$51, $$46, $$47][$$50, $$48, $$54] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$54, $$49, $$50][$$53, $$51, $$57] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$54, $$49, $$50] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (`loj-super-key_02`.PartSupp) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$48, $$54] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$53, $$51, $$57] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_05.plan
index b9b6128..2e2b4c8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_05.plan
@@ -3,19 +3,19 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$31(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$29][$$34] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$31][$$36] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.DS2) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$36] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_06.plan
index 4929664..326507f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_06.plan
@@ -3,15 +3,17 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$33(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$33(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$35(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$35(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DS1.DS1) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.DS2) |PARTITIONED|
+ -- STABLE_SORT [$$35(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DS2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_07.plan
index ee856a1..fafbbb2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_07.plan
@@ -3,16 +3,18 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$31(ASC) ] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$33(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$33(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.DS2.DS2) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$33(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.DS1) |PARTITIONED|
+ -- BTREE_SEARCH (test.DS2.DS2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$35(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DS1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_11.plan
index 4929664..326507f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/indexes_on_dataset_with_meta_11.plan
@@ -3,15 +3,17 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$33(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$33(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$35(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$35(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.DS1.DS1) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.DS2) |PARTITIONED|
+ -- STABLE_SORT [$$35(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DS2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/ASTERIXDB-2199.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/ASTERIXDB-2199.plan
index 1df1923..9d2ca53 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/ASTERIXDB-2199.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/ASTERIXDB-2199.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$34][$$35] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$36][$$37] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$36] |PARTITIONED|
-- DATASOURCE_SCAN (Facebook.Friendship) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- DATASOURCE_SCAN (Facebook.Friendship) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
index 8b6b284..0aa8dc4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
@@ -4,25 +4,25 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
- -- RANGE_PARTITION_EXCHANGE [$$86(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$89(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$89(ASC)] |PARTITIONED|
-- FORWARD |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$71] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$74] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$71(ASC), $$72(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
+ -- STABLE_SORT [$$74(ASC), $$75(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$92][$$71] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$95][$$74] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$95] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -34,7 +34,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -47,7 +47,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$104(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$107(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
@@ -103,19 +103,19 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$71] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$74] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$71(ASC), $$72(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
+ -- STABLE_SORT [$$74(ASC), $$75(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$92][$$71] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$95][$$74] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$95] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -127,7 +127,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -140,7 +140,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$104(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$107(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
index e6b48bb..f9d1b9d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
@@ -4,25 +4,25 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
- -- RANGE_PARTITION_EXCHANGE [$$86(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$89(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$89(ASC)] |PARTITIONED|
-- FORWARD |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$71] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$74] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$71(ASC), $$72(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
+ -- STABLE_SORT [$$74(ASC), $$75(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$92][$$71] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$95][$$74] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$95] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -34,7 +34,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -45,7 +45,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$98(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$101(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
@@ -68,19 +68,19 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$71] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$74] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$71(ASC), $$72(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
+ -- STABLE_SORT [$$74(ASC), $$75(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$92][$$71] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$95][$$74] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$95] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -92,7 +92,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -103,7 +103,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$98(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$101(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.topicKeywordIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
index 8b6b284..0aa8dc4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
@@ -4,25 +4,25 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
- -- RANGE_PARTITION_EXCHANGE [$$86(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$89(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$89(ASC)] |PARTITIONED|
-- FORWARD |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$71] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$74] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$71(ASC), $$72(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
+ -- STABLE_SORT [$$74(ASC), $$75(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$92][$$71] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$95][$$74] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$95] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -34,7 +34,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -47,7 +47,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$104(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$107(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
@@ -103,19 +103,19 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$71] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$74] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$71(ASC), $$72(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
+ -- STABLE_SORT [$$74(ASC), $$75(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$92][$$71] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$95][$$74] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$95] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -127,7 +127,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -140,7 +140,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$104(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$107(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
index 0cee768..e75d34e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
@@ -4,10 +4,10 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$66] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$71] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
- -- MICRO_PRE_CLUSTERED_GROUP_BY[$$67] |LOCAL|
+ -- MICRO_PRE_CLUSTERED_GROUP_BY[$$72] |LOCAL|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
@@ -17,24 +17,24 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$66(ASC), $$67(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$66] |PARTITIONED|
+ -- STABLE_SORT [$$71(ASC), $$72(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$67][$$68] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$67] |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$66][$$70] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$72][$$73] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$72] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$71][$$75] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
-- DATASOURCE_SCAN (tpch.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$70] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$75] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$68] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$73] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
index acca388..86e90b4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
@@ -3,25 +3,25 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$69(ASC)] |PARTITIONED|
- -- RANGE_PARTITION_EXCHANGE [$$69(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$72(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$72(ASC)] |PARTITIONED|
-- FORWARD |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$59] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$62] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC), $$60(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC), $$63(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$75][$$59] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$78][$$62] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$78] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -31,7 +31,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -42,7 +42,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$83(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
@@ -90,19 +90,19 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$59] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$62] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$59(ASC), $$60(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC), $$63(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$75][$$59] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$78][$$62] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$78] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -112,7 +112,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$59] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -123,7 +123,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (test.TweetMessages.TweetMessages) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$83(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$86(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH (test.TweetMessages.msgNgramIx) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
index f94167e..6bbed20 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
@@ -16,17 +16,19 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
index e20acdb..4c73e76 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
@@ -16,17 +16,19 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- SORT_MERGE_EXCHANGE [$$26(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
+ -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
index f94167e..6bbed20 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
@@ -16,17 +16,19 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
index e20acdb..4c73e76 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
@@ -16,17 +16,19 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- SORT_MERGE_EXCHANGE [$$26(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
+ -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (tpch.Orders.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (tpch.Orders.idx_Orders_Custkey) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/delete-primary-key-index-with-secondary.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/delete-primary-key-index-with-secondary.plan
index 6c4ca95..b357af8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/delete-primary-key-index-with-secondary.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/delete-primary-key-index-with-secondary.plan
@@ -10,7 +10,7 @@
-- INSERT_DELETE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- MATERIALIZE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$16] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan
index f22ba51..156c096 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan
@@ -4,23 +4,23 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$193][$$221] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$205][$$233] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (q08_group_by.Supplier) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$221] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$233] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$218][$$200] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$218] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$230][$$212] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$230] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$194][$$196] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$194] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$206][$$208] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$206] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -28,11 +28,11 @@
-- DATASOURCE_SCAN (q08_group_by.LineItem) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$196] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$208] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$209][$$197] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$209] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$221][$$209] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$221] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -40,36 +40,36 @@
-- DATASOURCE_SCAN (q08_group_by.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$197] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$209] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$206][$$198] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$206] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$218][$$210] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$218] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (q08_group_by.Customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$198] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$210] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$204][$$199] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$204] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$216][$$211] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$216] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (q08_group_by.Nation) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$211] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (q08_group_by.Region) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$212] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan
index 9bc4351..fde2a09 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan
@@ -4,48 +4,48 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$139][$$161] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$147][$$169] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$147] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (q09_group_by.Part) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$161] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$169] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$140, $$141][$$161, $$146] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$148, $$149][$$169, $$154] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$148, $$149] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (q09_group_by.Partsupp) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$161, $$146] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$169, $$154] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$142][$$146] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$142] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$150][$$154] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$150] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$149][$$143] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$149] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$157][$$151] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$157] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (q09_group_by.Supplier) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$151] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (q09_group_by.Nation) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$146] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$154] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan
index 6c713c4..6435a29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan
@@ -3,31 +3,26 @@
-- STREAM_LIMIT |UNPARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$45(DESC), $$46(DESC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$47(DESC), $$48(DESC) ] |PARTITIONED|
-- STREAM_LIMIT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [topK: 5] [$$45(DESC), $$46(DESC)] |PARTITIONED|
+ -- STABLE_SORT [topK: 5] [$$47(DESC), $$48(DESC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$45][$$54] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$47][$$56] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TinySocial.FacebookUsers) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TinySocial.FacebookUsers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$56] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- SUBPLAN |PARTITIONED|
@@ -39,18 +34,16 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- NESTED_LOOP |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- DATASOURCE_SCAN (TinySocial.FacebookUsers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TinySocial.FacebookUsers) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408.plan
index 9a5ddbf..0993f9b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408.plan
@@ -3,32 +3,34 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$45(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$43] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$46] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$43][$$46] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.customers) |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$46][$$49] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- DATASOURCE_SCAN (test.customers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408_ps.plan
index 8456246..f2e47fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408_ps.plan
@@ -4,38 +4,40 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
- -- RANGE_PARTITION_EXCHANGE [$$45(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$48(ASC)] |PARTITIONED|
-- FORWARD |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$43] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$46] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$43][$$46] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.customers) |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$46][$$49] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- DATASOURCE_SCAN (test.customers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
-- AGGREGATE |UNPARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
@@ -46,27 +48,29 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$43] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$46] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$43][$$46] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.customers) |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$46][$$49] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- DATASOURCE_SCAN (test.customers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue562_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue562_ps.plan
index 8230bf0..ea6589c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue562_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue562_ps.plan
@@ -9,12 +9,12 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$166] |PARTITIONED|
+ -- SORT_GROUP_BY[$$176] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$166] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$176] |PARTITIONED|
-- SORT_GROUP_BY[$$phone_substr] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -27,30 +27,30 @@
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$163] |PARTITIONED|
+ -- SORT_GROUP_BY[$$173] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$163] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$159] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$173] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$169] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$159(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$169(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$146][$$153] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$156][$$163] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$156] |PARTITIONED|
-- RUNNING_AGGREGATE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$phone_substr][$$148] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$phone_substr][$$158] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -61,7 +61,7 @@
-- BROADCAST_EXCHANGE |PARTITIONED|
-- UNNEST |UNPARTITIONED|
-- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$153] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$163] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -76,12 +76,12 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$166] |PARTITIONED|
+ -- SORT_GROUP_BY[$$176] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$166] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$176] |PARTITIONED|
-- SORT_GROUP_BY[$$phone_substr] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -94,30 +94,30 @@
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$163] |PARTITIONED|
+ -- SORT_GROUP_BY[$$173] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$163] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$159] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$173] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$169] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$159(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$169(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$146][$$153] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$156][$$163] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$156] |PARTITIONED|
-- RUNNING_AGGREGATE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$phone_substr][$$148] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$phone_substr][$$158] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -128,7 +128,7 @@
-- BROADCAST_EXCHANGE |PARTITIONED|
-- UNNEST |UNPARTITIONED|
-- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$153] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$163] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
index 4e9ed38..e23c4e5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
@@ -21,7 +21,7 @@
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- HYBRID_HASH_JOIN [$$47][$$49] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- DATASOURCE_SCAN (test.s) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
index 3a4e8a7..9f9e278 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
@@ -25,7 +25,7 @@
-- ASSIGN |UNPARTITIONED|
-- UNNEST |UNPARTITIONED|
-- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.s) |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rightouterjoin/roj-01-core.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rightouterjoin/roj-01-core.1.plan
index 2a8e400..e815859 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rightouterjoin/roj-01-core.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rightouterjoin/roj-01-core.1.plan
@@ -9,8 +9,8 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$85][$$86] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$91][$$92] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$91] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$92] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rightouterjoin/roj-01-core.2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rightouterjoin/roj-01-core.2.plan
index 2555179..d7c9f38 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rightouterjoin/roj-01-core.2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rightouterjoin/roj-01-core.2.plan
@@ -9,8 +9,8 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$142][$$143] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$151][$$152] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$151] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -25,8 +25,8 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$143][$$144] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$152][$$153] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$152] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -34,7 +34,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$153] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-index-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-index-01.plan
index 0d47f5c..d72e195 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-index-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-index-01.plan
@@ -2,18 +2,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.employee.employee) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$37(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.employee.employee) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$38(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.employee.idx_employee_fname) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (test.employee.idx_employee_fname) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-index-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-index-02.plan
index 3febc19..1697959 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-index-02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-index-02.plan
@@ -2,29 +2,31 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.employee.employee) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$31(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- INTERSECT |PARTITIONED|
+ -- BTREE_SEARCH (test.employee.employee) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$37(ASC)] |PARTITIONED|
+ -- INTERSECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$38(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.employee.idx_employee_fname) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$41(ASC)] |PARTITIONED|
+ -- BTREE_SEARCH (test.employee.idx_employee_fname) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.employee.idx_employee_lname) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BTREE_SEARCH (test.employee.idx_employee_lname) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in.plan
index bac3df2..79e304c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in.plan
@@ -2,30 +2,30 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$53(ASC) ] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$48] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$45] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$45][$$34] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$48][$$37] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$48] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan
index 540e821..162e3b4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan
@@ -2,35 +2,35 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$55(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$55(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$52] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- MICRO_PRE_CLUSTERED_GROUP_BY[] |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$49] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- UNNEST |LOCAL|
- -- MICRO_PRE_CLUSTERED_GROUP_BY[] |LOCAL|
- {
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
+ -- STABLE_SORT [$$52(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$49][$$48] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$52][$$51] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$52] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$48] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$51] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated_ps.plan
index 6080780..80134f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated_ps.plan
@@ -3,8 +3,8 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$55(ASC)] |PARTITIONED|
- -- RANGE_PARTITION_EXCHANGE [$$55(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$58(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$58(ASC)] |PARTITIONED|
-- FORWARD |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
@@ -12,7 +12,7 @@
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$49] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$52] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
@@ -26,22 +26,24 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$49][$$48] |PARTITIONED|
+ -- STABLE_SORT [$$52(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$52][$$51] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$52] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$48] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$51] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Orders) |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
-- AGGREGATE |UNPARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
@@ -52,7 +54,7 @@
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$49] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$52] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
@@ -66,19 +68,21 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$49][$$48] |PARTITIONED|
+ -- STABLE_SORT [$$52(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$52][$$51] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$52] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$48] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$51] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Orders) |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_8.plan
index 878c508..641a491 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_8.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_let_8.plan
@@ -2,30 +2,30 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$101(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$101(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$107(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$107(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$94, $$i1, $$97, $$93, $$104] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$100, $$i1, $$103, $$99, $$110] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$94(ASC), $$i1(ASC), $$97(ASC), $$93(ASC), $$104(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$100(ASC), $$i1(ASC), $$103(ASC), $$99(ASC), $$110(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$97][$$pid] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$97] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$103][$$pid] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$103] |PARTITIONED|
-- WINDOW_STREAM |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$94(ASC), $$i1(ASC), $$97(ASC), $$93(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$100(ASC), $$i1(ASC), $$103(ASC), $$99(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$100, $$i1, $$103, $$99] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- UNNEST |PARTITIONED|
@@ -44,13 +44,13 @@
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$112] |PARTITIONED|
+ -- SORT_GROUP_BY[$$118] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$112] |PARTITIONED|
- -- SORT_GROUP_BY[$$92] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$118] |PARTITIONED|
+ -- SORT_GROUP_BY[$$98] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
@@ -69,4 +69,4 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.cart) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_ps.plan
index b92fa1f..615f0c0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_ps.plan
@@ -3,8 +3,8 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
- -- RANGE_PARTITION_EXCHANGE [$$50(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$53(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$53(ASC)] |PARTITIONED|
-- FORWARD |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
@@ -12,31 +12,33 @@
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$45] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$48] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$45][$$34] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$48][$$37] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$48] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Orders) |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
-- AGGREGATE |UNPARTITIONED|
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
@@ -47,28 +49,30 @@
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$45] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$48] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$45][$$34] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$48][$$37] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$48] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Orders) |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Customers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815-2.plan
index 897163e..cb4bb4f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815-2.plan
@@ -2,71 +2,73 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$82(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$82(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$87(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$87(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$78] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$83] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$83(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$78][$$89] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$83][$$94] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.RawTweet) |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- WINDOW_STREAM |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$89(ASC), $$e.url(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$89] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$ve][$$81] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$ve] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- UNNEST |PARTITIONED|
+ -- DATASOURCE_SCAN (test.RawTweet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- WINDOW_STREAM |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$94(ASC), $$e.url(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$94] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$ve][$$86] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$ve] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$87][$$83] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$87] |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$92][$$88] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$92] |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.RawTweet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$88] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.RawTweet) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Verification) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Verification) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$86] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Evidence) |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Evidence) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815-3.plan
index 6ca6f1c..924d087 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815-3.plan
@@ -2,72 +2,74 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$88(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$88(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$93(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$93(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$84] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$89] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$89(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$84][$$95] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$89][$$100] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$89] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.RawTweet) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$95] |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- WINDOW_STREAM |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$72(ASC), $$95(ASC), $$83(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$72, $$95] |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$ve][$$87] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$ve] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.RawTweet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$100] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- WINDOW_STREAM |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$77(ASC), $$100(ASC), $$88(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$77, $$100] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$ve][$$92] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$ve] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$93][$$89] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$93] |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$98][$$94] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$98] |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.RawTweet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$94] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.RawTweet) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$89] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Verification) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$87] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Verification) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$92] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Evidence) |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Evidence) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815.plan
index 363b2bd..9aba5cf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815.plan
@@ -2,73 +2,75 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$75(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$75(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$80(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$80(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$70] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$75] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$75(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$70][$$82] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$75][$$87] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$75] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.RawTweet) |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$82] |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- DATASOURCE_SCAN (test.RawTweet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$87] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$62(ASC), $$82(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$62, $$82] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$ve][$$73] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$ve] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- UNNEST |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$67(ASC), $$87(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$67, $$87] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$ve][$$78] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$ve] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$80][$$76] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$80] |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$85][$$81] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$85] |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.RawTweet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.RawTweet) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$76] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Verification) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$73] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Verification) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$78] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.Evidence) |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.Evidence) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan
index 15e7546..985399f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan
@@ -2,21 +2,21 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$277(ASC), $#4(ASC), $#5(ASC), $#6(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$277(ASC), $#4(ASC), $#5(ASC), $#6(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$296(ASC), $#4(ASC), $#5(ASC), $#6(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$296(ASC), $#4(ASC), $#5(ASC), $#6(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$277, $$240, $$290, $$287, $#4, $$295][$$303, $$308, $$298, $$299, $$300, $$301] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$296, $$259, $$309, $$306, $#4, $$314][$$322, $$327, $$317, $$318, $$319, $$320] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- WINDOW_STREAM |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$277(ASC), $$240(ASC), $$290(ASC), $$287(ASC), $#4(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$296(ASC), $$259(ASC), $$309(ASC), $$306(ASC), $#4(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$277][$$jid] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$296][$$jid] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$296] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -32,13 +32,13 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$324, $$325] |PARTITIONED|
+ -- SORT_GROUP_BY[$$343, $$344] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$324, $$325] |PARTITIONED|
- -- SORT_GROUP_BY[$$273, $$274] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$343, $$344] |PARTITIONED|
+ -- SORT_GROUP_BY[$$292, $$293] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
@@ -62,12 +62,12 @@
-- STREAM_SELECT |PARTITIONED|
-- WINDOW_STREAM |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$303(ASC), $$308(ASC), $$298(ASC), $$299(ASC), $$300(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$322(ASC), $$327(ASC), $$317(ASC), $$318(ASC), $$319(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$303][$$306] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$322][$$325] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$322] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
@@ -75,18 +75,18 @@
-- DATASOURCE_SCAN (test.jds) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$306] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$325] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$327, $$328] |PARTITIONED|
+ -- SORT_GROUP_BY[$$346, $$347] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$327, $$328] |PARTITIONED|
- -- SORT_GROUP_BY[$$311, $$312] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$346, $$347] |PARTITIONED|
+ -- SORT_GROUP_BY[$$330, $$331] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
@@ -108,13 +108,13 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$330, $$331] |PARTITIONED|
+ -- SORT_GROUP_BY[$$349, $$350] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$330, $$331] |PARTITIONED|
- -- SORT_GROUP_BY[$$275, $$276] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$349, $$350] |PARTITIONED|
+ -- SORT_GROUP_BY[$$294, $$295] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1580.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1580.plan
index 71d5343..cbeb55d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1580.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1580.plan
@@ -3,21 +3,21 @@
-- STREAM_LIMIT |UNPARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$128(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$136(ASC) ] |PARTITIONED|
-- STREAM_LIMIT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [topK: 100] [$$128(ASC)] |PARTITIONED|
+ -- STABLE_SORT [topK: 100] [$$136(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$137] |PARTITIONED|
+ -- SORT_GROUP_BY[$$145] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
- -- SORT_GROUP_BY[$$118] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$145] |PARTITIONED|
+ -- SORT_GROUP_BY[$$126] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
@@ -25,33 +25,33 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$121][$$124] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$121] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$129][$$132] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$129] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$129][$$123] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$129] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$137][$$131] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$120][$$134] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$120] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$128][$$142] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$128] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$119][$$132] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$127][$$140] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$127] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.customer_address) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$132] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$140] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$134] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$142] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -59,13 +59,13 @@
-- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.date_dim) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$132] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.item) |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated-2.plan
index bda21dc..b010fca 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated-2.plan
@@ -8,7 +8,7 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$227] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$234] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- AGGREGATE |LOCAL|
@@ -16,25 +16,25 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$234(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$227][$$175] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$189] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
+ -- HYBRID_HASH_JOIN [$$234][$$182] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$189(ASC)] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$196] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$189][$$225] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$196][$$232] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$137] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$144] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- AGGREGATE |LOCAL|
@@ -42,134 +42,142 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$144(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$137][$$187] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$187] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$144][$$194] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$144] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$233, $$234] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- HASH_PARTITION_EXCHANGE [$$233, $$234] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$181, $$186] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$194] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$240, $$241] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$181(ASC), $$186(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$240, $$241] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$188, $$193] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$188(ASC), $$193(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$170][$$169] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$204] |PARTITIONED|
- -- RUNNING_AGGREGATE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$177][$$176] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$211] |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$208] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$215] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$208][$$209] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$215(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$207] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$215][$$216] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$215] |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$214] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$169] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$176] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$225] |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$232] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$239, $$240] |PARTITIONED|
+ -- SORT_GROUP_BY[$$246, $$247] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$239, $$240] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$191, $$224] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$246, $$247] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$198, $$231] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$191(ASC), $$224(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$198(ASC), $$231(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$174][$$173] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$174] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$181][$$180] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$181] |PARTITIONED|
-- RUNNING_AGGREGATE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$194] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$201] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- AGGREGATE |LOCAL|
@@ -177,58 +185,60 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$194(ASC)] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$201][$$178] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$194][$$171] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$202] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$195] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
+ -- STABLE_SORT [$$202(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$195][$$197] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$202][$$204] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$215] |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$197] |PARTITIONED|
+ -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$204] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$236, $$237] |PARTITIONED|
+ -- SORT_GROUP_BY[$$243, $$244] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$236, $$237] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$202, $$203] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$243, $$244] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$209, $$210] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$202(ASC), $$203(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$209(ASC), $$210(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$204][$$207] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$211][$$214] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$204] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$211] |PARTITIONED|
-- RUNNING_AGGREGATE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -237,7 +247,7 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$208] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$215] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- AGGREGATE |LOCAL|
@@ -245,37 +255,41 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$215(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$208][$$209] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$215][$$216] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$207] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$215] |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$214] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$207] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$214] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -288,25 +302,25 @@
-- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$207] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$214] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$173] |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$180] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -319,21 +333,21 @@
-- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$207] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$214] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan
index d7bad18..de951d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan
@@ -16,19 +16,19 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$180(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$180][$$222] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$164] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
+ -- HYBRID_HASH_JOIN [$$180][$$222] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$164(ASC)] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$164] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -42,37 +42,41 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$144(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$144][$$154] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$144][$$154] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$144] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- HASH_PARTITION_EXCHANGE [$$178] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -118,34 +122,36 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$206(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$206][$$207] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$206][$$207] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$206] |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- HASH_PARTITION_EXCHANGE [$$156] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -157,47 +163,45 @@
-- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$222] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$222] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$233, $$234] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- HASH_PARTITION_EXCHANGE [$$233, $$234] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$181, $$184] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$233, $$234] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$181(ASC), $$184(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$233, $$234] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$181, $$184] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$181(ASC), $$184(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$159][$$158] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$159] |PARTITIONED|
- -- RUNNING_AGGREGATE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$159][$$158] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$159] |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$185] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$185(ASC)] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$185] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -216,34 +220,36 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$206(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$206][$$207] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$206][$$207] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$206] |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- HASH_PARTITION_EXCHANGE [$$188] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -285,34 +291,36 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$206(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$206][$$207] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$206][$$207] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$206] |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- BTREE_SEARCH (tpcds.item.item) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
@@ -328,16 +336,16 @@
-- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$158] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$158] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1591.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1591.plan
index c57a9b4..bdf16c1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1591.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1591.plan
@@ -4,91 +4,91 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$184(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$192(ASC) ] |PARTITIONED|
-- STREAM_LIMIT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$193, $$194] |PARTITIONED|
+ -- SORT_GROUP_BY[$$201, $$202] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$193, $$194] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$181, $$182] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$201, $$202] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$189, $$190] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$181(ASC), $$182(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$189(ASC), $$190(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$181][$$163] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$181] |PARTITIONED|
- -- SORT_GROUP_BY[$$190, $$191] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$189][$$171] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$189] |PARTITIONED|
+ -- SORT_GROUP_BY[$$198, $$199] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$190, $$191] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$178, $$179] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$198, $$199] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$186, $$187] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$178(ASC), $$179(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$186(ASC), $$187(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$178][$$161] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$178] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$186][$$169] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$186] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$187, $$188] |PARTITIONED|
+ -- SORT_GROUP_BY[$$195, $$196] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$187, $$188] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$142, $$143] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$195, $$196] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$150, $$151] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$142(ASC), $$143(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$150(ASC), $$151(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$142][$$159] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$142] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$150][$$167] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$150] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$153][$$143] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$153] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$161][$$151] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$161] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$151] |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.customer_address) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$159] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$167] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$164][$$146] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$164] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$172][$$154] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$172] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -97,18 +97,18 @@
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$154] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.date_dim) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$161] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$169] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$166][$$149] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$166] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$174][$$157] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$174] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -120,18 +120,18 @@
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$154] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.date_dim) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$163] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$171] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$168][$$152] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$168] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$176][$$160] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$176] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -143,7 +143,7 @@
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$154] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.date_dim) |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1596.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1596.plan
index d27745b..6b55682 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1596.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1596.plan
@@ -3,25 +3,25 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$50(ASC), $$51(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$50(ASC), $$51(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$53(ASC), $$54(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$53(ASC), $$54(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$50][$$54] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$50] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$53][$$57] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$51, $$50][$$53, $$52] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$54, $$53][$$56, $$55] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$54, $$53] |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.catalog_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$56, $$55] |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.catalog_returns) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$57] |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.item) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1596_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1596_ps.plan
index e83f7f4..ea90224 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1596_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1596_ps.plan
@@ -4,28 +4,28 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$50(ASC), $$51(ASC)] |PARTITIONED|
- -- RANGE_PARTITION_EXCHANGE [$$50(ASC), $$51(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$53(ASC), $$54(ASC)] |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$53(ASC), $$54(ASC)] |PARTITIONED|
-- FORWARD |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$50][$$54] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$50] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$53][$$57] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$51, $$50][$$53, $$52] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$54, $$53][$$56, $$55] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$54, $$53] |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.catalog_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$56, $$55] |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.catalog_returns) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$57] |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.item) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
@@ -39,20 +39,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$50][$$54] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$50] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$53][$$57] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$51, $$50][$$53, $$52] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$54, $$53][$$56, $$55] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$54, $$53] |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.catalog_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$56, $$55] |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.catalog_returns) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$57] |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.item) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan
index 0be7ab2..cd5de62 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan
@@ -3,13 +3,13 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ] |PARTITIONED|
- -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$135] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
- -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$135] |PARTITIONED|
+ -- SORT_GROUP_BY[$$118] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
@@ -17,12 +17,12 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$124] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$122][$$118] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$122] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$126][$$122] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$126] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -31,7 +31,7 @@
-- DATASOURCE_SCAN (tpch.LineItem) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$122] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_ps.plan
index bbd6cf0..03a4f96 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_ps.plan
@@ -9,13 +9,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$135] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
- -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$135] |PARTITIONED|
+ -- SORT_GROUP_BY[$$118] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
@@ -23,12 +23,12 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$124] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$122][$$118] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$122] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$126][$$122] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$126] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -37,7 +37,7 @@
-- DATASOURCE_SCAN (tpch.LineItem) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$122] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -55,13 +55,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$135] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
- -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$135] |PARTITIONED|
+ -- SORT_GROUP_BY[$$118] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
@@ -69,12 +69,12 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$124] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$122][$$118] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$122] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$126][$$122] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$126] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -83,7 +83,7 @@
-- DATASOURCE_SCAN (tpch.LineItem) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$122] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/btree-index-composite-key-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/btree-index-composite-key-04.plan
index ce1677d..0e5a3a7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/btree-index-composite-key-04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/btree-index-composite-key-04.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.employee) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$22(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$22(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.employee) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-02.plan
index a41ba60..c91805c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-02.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$59(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds2) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$61(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-10.plan
index 753d23d..62d8cf1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-10.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-11.plan
index 753d23d..62d8cf1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-11.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-12.plan
index 753d23d..62d8cf1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-12.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-12.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-13.plan
index f6647d4..607e59c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-13.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-13.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-14.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-14.plan
index f6647d4..607e59c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-14.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-14.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-15.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-15.plan
index f6647d4..607e59c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-15.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-15.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$48(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$50(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-20.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-20.plan
index e5fd2d4..37bd6b8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-20.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-20.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$60(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$60(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-23.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-23.plan
index e5fd2d4..37bd6b8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-23.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-23.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$60(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$60(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-24.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-24.plan
index c587ffb..12adbc6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-24.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-24.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-25.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-25.plan
index e5fd2d4..37bd6b8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-25.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/cast-default-null/cast-default-null-25.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$58(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$60(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$60(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01-disable-idxonly.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01-disable-idxonly.plan
index 8dd4216..8d3aa97 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01-disable-idxonly.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01-disable-idxonly.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.MyData) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.MyData) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01.plan
index 8dd4216..8d3aa97 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index/btree-sidx-idxonly-01.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.MyData) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.MyData) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q16.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q16.plan
index 2214a1e..fab0c7b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q16.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q16.plan
@@ -2,42 +2,42 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$131(DESC) ] |PARTITIONED|
- -- STABLE_SORT [$$131(DESC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$139(DESC) ] |PARTITIONED|
+ -- STABLE_SORT [$$139(DESC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$121, $$122, $$123] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$129, $$130, $$131] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- MICRO_PRE_SORTED_DISTINCT_BY |LOCAL|
- -- MICRO_STABLE_SORT [$$142(ASC)] |LOCAL|
+ -- MICRO_STABLE_SORT [$$150(ASC)] |LOCAL|
-- ASSIGN |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$121(ASC), $$122(ASC), $$123(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$121, $$122, $$123] |PARTITIONED|
+ -- STABLE_SORT [$$129(ASC), $$130(ASC), $$131(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$129, $$130, $$131] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$125, $$126] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$133, $$134] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$125(ASC), $$126(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$133(ASC), $$134(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$133, $$134] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- NESTED_LOOP |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$130][$$129] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$138][$$137] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-basic/ngram-contains.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-basic/ngram-contains.plan
index 8a9e3fa..da22292 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-basic/ngram-contains.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/inverted-index-basic/ngram-contains.plan
@@ -1,9 +1,11 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$15(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$16(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DBLP) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_05.plan
index 6013ad8..e8d4f16 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_05.plan
@@ -3,16 +3,18 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$31(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (test.DS2.DS2) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$34(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.DS1) |PARTITIONED|
+ -- BTREE_SEARCH (test.DS2.DS2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$36] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.DS1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-index/btree-index-join/ASTERIXDB-2199.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-index/btree-index-join/ASTERIXDB-2199.plan
index 46dbf81..9d2ca53 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-index/btree-index-join/ASTERIXDB-2199.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-index/btree-index-join/ASTERIXDB-2199.plan
@@ -5,11 +5,11 @@
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- HYBRID_HASH_JOIN [$$36][$$37] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$36] |PARTITIONED|
-- DATASOURCE_SCAN (Facebook.Friendship) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- DATASOURCE_SCAN (Facebook.Friendship) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01.plan
index e4167f0..cf73858 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01.plan
@@ -16,10 +16,12 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02.plan
index c87d9ce..b57024b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02.plan
@@ -16,10 +16,12 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- SORT_MERGE_EXCHANGE [$$26(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01.plan
index e4167f0..cf73858 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01.plan
@@ -16,10 +16,12 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02.plan
index c87d9ce..b57024b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02.plan
@@ -16,10 +16,12 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- SORT_MERGE_EXCHANGE [$$26(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (tpch.Orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q08_group_by.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q08_group_by.plan
index 1e558ec..2b4df2f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q08_group_by.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q08_group_by.plan
@@ -25,7 +25,7 @@
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- HYBRID_HASH_JOIN [$$205][$$233] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$205] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -40,7 +40,7 @@
-- DATASOURCE_SCAN (q08_group_by.LineItem) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$208] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -48,28 +48,28 @@
-- DATASOURCE_SCAN (q08_group_by.Orders) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$209] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (q08_group_by.Customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$210] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (q08_group_by.Nation) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$211] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (q08_group_by.Region) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$212] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q09_group_by.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q09_group_by.plan
index 26a56eb..ec25577 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q09_group_by.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q09_group_by.plan
@@ -17,7 +17,7 @@
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- HYBRID_HASH_JOIN [$$147][$$169] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$147] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -32,21 +32,21 @@
-- DATASOURCE_SCAN (q09_group_by.LineItem) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$148, $$149] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (q09_group_by.Partsupp) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$150] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (q09_group_by.Supplier) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$151] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-index-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-index-01.plan
index cecfb1c..3bb3b7d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-index-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-index-01.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.employee) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.employee) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-index-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-index-02.plan
index cce5aaa..863378a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-index-02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/statement-params/statement-params-index-02.plan
@@ -2,11 +2,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.employee) |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$31(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.employee) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1580.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1580.plan
index eb98d7c..cbeb55d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1580.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1580.plan
@@ -38,7 +38,7 @@
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- HYBRID_HASH_JOIN [$$127][$$140] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$127] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.customer_address) |PARTITIONED|
@@ -59,13 +59,13 @@
-- DATASOURCE_SCAN (tpcds.store_sales) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.date_dim) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$132] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.item) |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1591.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1591.plan
index d3fce85..bdf16c1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1591.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1591.plan
@@ -79,7 +79,7 @@
-- DATASOURCE_SCAN (tpcds.customer) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$151] |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.customer_address) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
@@ -97,7 +97,7 @@
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$154] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.date_dim) |PARTITIONED|
@@ -120,7 +120,7 @@
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$154] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.date_dim) |PARTITIONED|
@@ -143,7 +143,7 @@
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$154] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (tpcds.date_dim) |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping.plan
index 7609856..f1672b9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping.plan
@@ -3,13 +3,13 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ] |PARTITIONED|
- -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$135] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
- -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$135] |PARTITIONED|
+ -- SORT_GROUP_BY[$$118] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$118][$$122] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$122][$$126] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -28,7 +28,7 @@
-- BROADCAST_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$124] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_ps.plan
index f21d402..922ecc4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_ps.plan
@@ -9,13 +9,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$135] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
- -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$135] |PARTITIONED|
+ -- SORT_GROUP_BY[$$118] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
@@ -23,7 +23,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$118][$$122] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$122][$$126] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -34,7 +34,7 @@
-- BROADCAST_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$124] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
@@ -55,13 +55,13 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- REPLICATE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$131] |PARTITIONED|
+ -- SORT_GROUP_BY[$$135] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$131] |PARTITIONED|
- -- SORT_GROUP_BY[$$114] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$135] |PARTITIONED|
+ -- SORT_GROUP_BY[$$118] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
@@ -69,7 +69,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$118][$$122] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$122][$$126] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -80,7 +80,7 @@
-- BROADCAST_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$114][$$120] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$124] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/group_only/group_only.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/group_only/group_only.3.query.sqlpp
index 6cde5e4..65149b7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/group_only/group_only.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/group_only/group_only.3.query.sqlpp
@@ -24,7 +24,7 @@
SELECT d.authors AS author, d.title AS title
FROM (SELECT VALUE t
FROM DBLP t
- LIMIT 5) AS d
+ ORDER BY id LIMIT 5) AS d
GROUP BY d.authors, d.title
ORDER BY d.authors;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/policy-05/policy-05.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/policy-05/policy-05.3.query.sqlpp
index b20900a..e0623e8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/policy-05/policy-05.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/policy-05/policy-05.3.query.sqlpp
@@ -34,5 +34,5 @@
)[0]
)
)[0] / (COUNT(*) * 1.0 ) AS risk
-ORDER BY risk DESC
+ORDER BY state ASC, risk DESC
LIMIT 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.query.sqlpp
index 212479a..dbc6633 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.query.sqlpp
@@ -23,4 +23,4 @@
select element c
from LineItem as c
where c.l_suppkey < 150 AND l_extendedprice < 10000
-limit 5 offset 5;
+order by c.l_orderkey,c.l_linenumber limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.6.query.sqlpp
index e7ac96d..a52a848 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.6.query.sqlpp
@@ -24,4 +24,4 @@
c.l_suppkey > 0 as suppkey
from LineItem as c
where c.l_suppkey < 150
-limit 5;
+order by c.l_orderkey,c.l_linenumber limit 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.query.sqlpp
index 026aed7..b2700ea 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.query.sqlpp
@@ -23,4 +23,4 @@
select element c
from LineItem as c
where (c.l_suppkey < 150)
-limit 5 offset 5;
+order by c.l_orderkey,c.l_linenumber limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.query.sqlpp
index 6459b5c..21518dc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.query.sqlpp
@@ -23,4 +23,4 @@
select element paper
from DBLP1 as paper
where contains(dblpid, 'kimL89')
-limit 5 offset 5;
+order by id limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.query.sqlpp
index 5b7c5be..4777739 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.query.sqlpp
@@ -22,4 +22,4 @@
select element paper
from DBLP1 as paper
-limit 5 offset 5;
+order by id limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp
index c4335af8..0d5b129 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp
@@ -59,6 +59,6 @@
"t8": (
select value object_remove(object_remove( object_remove(u, "lang"), "friends_count"), "statuses_count")
from TwitterUsers u
- limit 1
+ order by u.`screen-name` limit 1
)
};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.07.query.sqlpp
index e703f5d..0deeb10 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.07.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.07.query.sqlpp
@@ -27,6 +27,6 @@
WHERE v.review = "good"
AND year IS NOT UNKNOWN
AND quarter IS NOT UNKNOWN
- LIMIT 3
+ ORDER BY v.id LIMIT 3
) x
ORDER BY x.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.11.query.sqlpp
index acf950e..4cc31cd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.11.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-pushdown/view-pushdown.11.query.sqlpp
@@ -27,6 +27,6 @@
WHERE v.review = "good"
AND year IS NOT UNKNOWN
AND quarter IS NOT UNKNOWN
- LIMIT 3
+ ORDER BY v.id LIMIT 3
) x
ORDER BY x.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/rebalance.xml b/asterixdb/asterix-app/src/test/resources/runtimets/rebalance.xml
index 45077a9..511e27b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/rebalance.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/rebalance.xml
@@ -16,7 +16,8 @@
! specific language governing permissions and limitations
! under the License.
!-->
-<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp">
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results"
+ QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
<test-group name="rebalance">
<test-case FilePath="rebalance">
<compilation-unit name="duplicate_location">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/count_dataset/count_dataset.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/count_dataset/count_dataset.1.plan
index 58837ea..d332a4f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/count_dataset/count_dataset.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/count_dataset/count_dataset.1.plan
@@ -1,26 +1,30 @@
-distribute result [$$25] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+distribute result [$$26] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- aggregate [$$25] <- [agg-sql-sum($$28)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ aggregate [$$26] <- [agg-sql-sum($$29)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- AGGREGATE |UNPARTITIONED|
- aggregate [$$28] <- [agg-sql-count(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ aggregate [$$29] <- [agg-sql-count(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- AGGREGATE |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SORT_MERGE_EXCHANGE [$$26(ASC) ] |PARTITIONED|
- project ([$$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- select (and(ge($$24, 1), le($$24, 10))) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_SELECT |PARTITIONED|
- project ([$$26, $$24]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$27(ASC) ] |PARTITIONED|
+ order (ASC, $$27) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$27(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$27]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$24] <- [$$Tweet.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$26, $$Tweet] <- Twitter.Tweet [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ select (and(ge($$25, 1), le($$25, 10))) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$27, $$25]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$25] <- [$$Tweet.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/count_dataset/count_dataset.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/count_dataset/count_dataset.1.plan
index 584763d..09a6e7e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/count_dataset/count_dataset.1.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/count_dataset/count_dataset.1.plan
@@ -1,26 +1,30 @@
-distribute result [$$25] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+distribute result [$$26] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- aggregate [$$25] <- [agg-sum($$28)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ aggregate [$$26] <- [agg-sum($$29)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- AGGREGATE |UNPARTITIONED|
- aggregate [$$28] <- [agg-count(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ aggregate [$$29] <- [agg-count(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- AGGREGATE |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SORT_MERGE_EXCHANGE [$$26(ASC) ] |PARTITIONED|
- project ([$$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- select (and(ge($$24, 1), le($$24, 10))) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_SELECT |PARTITIONED|
- project ([$$26, $$24]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SORT_MERGE_EXCHANGE [$$27(ASC) ] |PARTITIONED|
+ order (ASC, $$27) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$27(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$27]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$24] <- [$$Tweet.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$26, $$Tweet] <- Twitter.Tweet [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ select (and(ge($$25, 1), le($$25, 10))) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$27, $$25]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$25] <- [$$Tweet.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.007.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.007.plan
index 2d8a975..24e6c63 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.007.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.007.plan
@@ -6,17 +6,21 @@
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- project ([$$d, $$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- select (and(gt($$19, "1"), lt($$19, "3"))) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_SELECT |PARTITIONED|
- assign [$$19] <- [$$d.getField("a")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$20, $$d] <- test.ColumnDataset filter on (and(gt($$19, "1"), lt($$19, "3"))) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ order (ASC, $$20) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$d, $$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ select (and(gt($$19, "1"), lt($$19, "3"))) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ assign [$$19] <- [$$d.getField("a")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$20, $$d] <- test.ColumnDataset filter on (and(gt($$19, "1"), lt($$19, "3"))) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.009.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.009.plan
index e9fa01c..a85812f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.009.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.009.plan
@@ -6,17 +6,21 @@
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- project ([$$d, $$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- select (and(gt($$19, 1), lt($$19, 3))) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_SELECT |PARTITIONED|
- assign [$$19] <- [$$d.getField("a")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$20, $$d] <- test.ColumnDataset filter on (and(gt($$19, 1), lt($$19, 3))) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ order (ASC, $$20) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$d, $$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ select (and(gt($$19, 1), lt($$19, 3))) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ assign [$$19] <- [$$d.getField("a")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$20, $$d] <- test.ColumnDataset filter on (and(gt($$19, 1), lt($$19, 3))) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.011.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.011.plan
index 776a360..c272cc1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.011.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.011.plan
@@ -6,30 +6,34 @@
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- SORT_MERGE_EXCHANGE [$$28(ASC) ] |PARTITIONED|
- project ([$$d, $$28]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- select ($$26) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_SELECT |PARTITIONED|
- project ([$$28, $$d, $$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ order (ASC, $$28) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$d, $$28]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- subplan {
- aggregate [$$26] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- AGGREGATE |LOCAL|
- select (lt($$a, "100")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_SELECT |LOCAL|
- unnest $$a <- scan-collection($$29) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- UNNEST |LOCAL|
- nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- NESTED_TUPLE_SOURCE |LOCAL|
- } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SUBPLAN |PARTITIONED|
- assign [$$29] <- [$$d.getField("array")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on (lt($$a, "100")) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ select ($$26) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$28, $$d, $$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ subplan {
+ aggregate [$$26] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ select (lt($$a, "100")) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_SELECT |LOCAL|
+ unnest $$a <- scan-collection($$29) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SUBPLAN |PARTITIONED|
+ assign [$$29] <- [$$d.getField("array")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on (lt($$a, "100")) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.013.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.013.plan
index d8b6f2d..6787c49 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.013.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.013.plan
@@ -6,30 +6,34 @@
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- SORT_MERGE_EXCHANGE [$$28(ASC) ] |PARTITIONED|
- project ([$$d, $$28]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- select ($$26) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_SELECT |PARTITIONED|
- project ([$$28, $$d, $$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ order (ASC, $$28) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$d, $$28]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- subplan {
- aggregate [$$26] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- AGGREGATE |LOCAL|
- select (lt($$a, 100)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_SELECT |LOCAL|
- unnest $$a <- scan-collection($$29) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- UNNEST |LOCAL|
- nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- NESTED_TUPLE_SOURCE |LOCAL|
- } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SUBPLAN |PARTITIONED|
- assign [$$29] <- [$$d.getField("array")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on (lt($$a, 100)) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ select ($$26) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$28, $$d, $$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ subplan {
+ aggregate [$$26] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ select (lt($$a, 100)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_SELECT |LOCAL|
+ unnest $$a <- scan-collection($$29) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SUBPLAN |PARTITIONED|
+ assign [$$29] <- [$$d.getField("array")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on (lt($$a, 100)) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.015.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.015.plan
index 35d8617..c592f2c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.015.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.015.plan
@@ -6,30 +6,34 @@
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- SORT_MERGE_EXCHANGE [$$28(ASC) ] |PARTITIONED|
- project ([$$d, $$28]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- select ($$26) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_SELECT |PARTITIONED|
- project ([$$28, $$d, $$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ order (ASC, $$28) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$d, $$28]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- subplan {
- aggregate [$$26] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- AGGREGATE |LOCAL|
- select (eq($$a, 100)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_SELECT |LOCAL|
- unnest $$a <- scan-collection($$29) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- UNNEST |LOCAL|
- nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- NESTED_TUPLE_SOURCE |LOCAL|
- } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SUBPLAN |PARTITIONED|
- assign [$$29] <- [$$d.getField("array")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on (eq($$a, 100)) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ select ($$26) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$28, $$d, $$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ subplan {
+ aggregate [$$26] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ select (eq($$a, 100)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_SELECT |LOCAL|
+ unnest $$a <- scan-collection($$29) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SUBPLAN |PARTITIONED|
+ assign [$$29] <- [$$d.getField("array")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on (eq($$a, 100)) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.017.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.017.plan
index 8071262..f61dee4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.017.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/filter/001/001.017.plan
@@ -6,30 +6,34 @@
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- SORT_MERGE_EXCHANGE [$$28(ASC) ] |PARTITIONED|
- project ([$$d, $$28]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- select ($$26) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_SELECT |PARTITIONED|
- project ([$$28, $$d, $$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ order (ASC, $$28) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$d, $$28]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- subplan {
- aggregate [$$26] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- AGGREGATE |LOCAL|
- select (lt($$a, 100.1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_SELECT |LOCAL|
- unnest $$a <- scan-collection($$29) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- UNNEST |LOCAL|
- nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- NESTED_TUPLE_SOURCE |LOCAL|
- } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SUBPLAN |PARTITIONED|
- assign [$$29] <- [$$d.getField("array")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on (lt($$a, 100.1)) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ select ($$26) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$28, $$d, $$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ subplan {
+ aggregate [$$26] <- [non-empty-stream()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ select (lt($$a, 100.1)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_SELECT |LOCAL|
+ unnest $$a <- scan-collection($$29) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SUBPLAN |PARTITIONED|
+ assign [$$29] <- [$$d.getField("array")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$28, $$d] <- test.ColumnDataset filter on (lt($$a, 100.1)) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.004.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.004.plan
index ad93872..01a6d74 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.004.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.004.plan
@@ -10,15 +10,19 @@
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- project ([$$22, $$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$22] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$20, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ order (ASC, $$20) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$22, $$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$20, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.006.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.006.plan
index bfe2674..250df7f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.006.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.006.plan
@@ -10,21 +10,25 @@
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
-- SORT_MERGE_EXCHANGE [$$25(ASC) ] |PARTITIONED|
- project ([$$27, $$25]) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$27] <- [array-star($$24).getField("display_url")] [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
- -- ASSIGN |PARTITIONED|
- select (not(is-missing($$24))) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
- -- STREAM_SELECT |PARTITIONED|
- project ([$$25, $$24]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$24] <- [$$p.getField("entities").getField("urls")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$25, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ order (ASC, $$25) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
+ exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$27, $$25]) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$27] <- [array-star($$24).getField("display_url")] [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ select (not(is-missing($$24))) [cardinality: 2.1, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$25, $$24]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$24] <- [$$p.getField("entities").getField("urls")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$25, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.008.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.008.plan
index 93291ce..968dd6b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.008.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.008.plan
@@ -10,19 +10,23 @@
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
- project ([$$urls, $$30]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- unnest $$urls <- scan-collection($$31) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- UNNEST |PARTITIONED|
- project ([$$30, $$31]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ order (ASC, $$30) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$urls, $$30]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$31] <- [$$p.getField("entities").getField("urls")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$30, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ unnest $$urls <- scan-collection($$31) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- UNNEST |PARTITIONED|
+ project ([$$30, $$31]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$31] <- [$$p.getField("entities").getField("urls")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$30, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.012.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.012.plan
index 4f9517f..40b1f9c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.012.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/array-access-pushdown/array-access-pushdown.012.plan
@@ -10,15 +10,19 @@
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
- project ([$$25, $$21]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$25] <- [$$p.getField("place").getField("bounding_box")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$21, $$p] <- test.ColumnDataset project ({place:{bounding_box:{coordinates:[[[any]]]}}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ order (ASC, $$21) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$25, $$21]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$25] <- [$$p.getField("place").getField("bounding_box")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$21, $$p] <- test.ColumnDataset project ({place:{bounding_box:{coordinates:[[[any]]]}}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.004.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.004.plan
index febeed7..f8b3ca1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.004.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.004.plan
@@ -6,9 +6,13 @@
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- SORT_MERGE_EXCHANGE [$$14(ASC) ] |PARTITIONED|
- data-scan []<-[$$14, $$p] <- test.ColumnDataset1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ order (ASC, $$14) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$14(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$14, $$p] <- test.ColumnDataset1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.006.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.006.plan
index 2abd384..4953c85 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.006.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.006.plan
@@ -19,7 +19,7 @@
join (eq($$33, $$34)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
-- HYBRID_HASH_JOIN [$$33][$$34] |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
data-scan []<-[$$33, $$p1] <- test.ColumnDataset1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
@@ -27,7 +27,7 @@
empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
project ([$$34]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.008.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.008.plan
index 0601e69..228edaf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.008.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.008.plan
@@ -21,7 +21,7 @@
join (eq($$33, $$34)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
-- HYBRID_HASH_JOIN [$$33][$$34] |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
project ([$$38, $$33]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
assign [$$38] <- [$$p1.getField("age")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
@@ -35,7 +35,7 @@
empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
project ([$$39, $$34]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
assign [$$39] <- [$$p2.getField("name")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.010.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.010.plan
index 997f157..6e213db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.010.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/field-access-pushdown/field-access-pushdown.010.plan
@@ -14,9 +14,13 @@
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
- data-scan []<-[$$21, $$p] <- test.ColumnDataset4 project ({user:{name:any,id:any}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ order (ASC, $$21) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$21, $$p] <- test.ColumnDataset4 project ({user:{name:any,id:any}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/heterogeneous-access-pushdown/heterogeneous-access-pushdown.004.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/heterogeneous-access-pushdown/heterogeneous-access-pushdown.004.plan
index 06f13d6..d43be61 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/heterogeneous-access-pushdown/heterogeneous-access-pushdown.004.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/heterogeneous-access-pushdown/heterogeneous-access-pushdown.004.plan
@@ -14,9 +14,13 @@
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
- data-scan []<-[$$24, $$p] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ order (ASC, $$24) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$24, $$p] <- test.ColumnDataset project ({arrayOrObject:<[{text:any}],{text:any}>}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.004.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.004.plan
index 9c7238f..ea56165 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.004.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.004.plan
@@ -23,7 +23,7 @@
join (eq($$63, $$62)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
-- HYBRID_HASH_JOIN [$$62][$$63] |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
data-scan []<-[$$62, $$p1] <- test.ColumnDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
@@ -31,7 +31,7 @@
empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$63] |PARTITIONED|
data-scan []<-[$$63, $$p2] <- test.RowDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.012.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.012.plan
index 9e4e7f1..162b6b0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.012.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/pushdown/other-pushdowns/other-pushdowns.012.plan
@@ -10,21 +10,25 @@
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- SORT_MERGE_EXCHANGE [$$25(ASC) ] |PARTITIONED|
- project ([$$27, $$25]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$27] <- [get-item($$24, 0).getField("display_url")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- select (eq(sql-count($$24), 1)) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_SELECT |PARTITIONED|
- project ([$$25, $$24]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$24] <- [$$p.getField("entities").getField("urls")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$25, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ order (ASC, $$25) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$27, $$25]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$27] <- [get-item($$24, 0).getField("display_url")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ select (eq(sql-count($$24), 1)) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$25, $$24]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$24] <- [$$p.getField("entities").getField("urls")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$25, $$p] <- test.ColumnDataset project ({entities:{urls:[{display_url:any}]}}) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan
index 99f36c1..e7f6361 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.04.plan
@@ -1,62 +1,62 @@
-distribute result [$$48] [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+distribute result [$$51] [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
-- DISTRIBUTE_RESULT |PARTITIONED|
exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$48]) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ project ([$$51]) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$48] <- [{"n_nationkey": $$55, "s_nationkey": $$53, "c_nationkey": $$52}] [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ assign [$$51] <- [{"n_nationkey": $$58, "s_nationkey": $$56, "c_nationkey": $$55}] [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
-- ASSIGN |PARTITIONED|
exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
- -- SORT_MERGE_EXCHANGE [$$55(ASC), $$53(ASC), $$52(ASC) ] |PARTITIONED|
- order (ASC, $$55) (ASC, $$53) (ASC, $$52) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
- -- STABLE_SORT [$$55(ASC), $$53(ASC), $$52(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$58(ASC), $$56(ASC), $$55(ASC) ] |PARTITIONED|
+ order (ASC, $$58) (ASC, $$56) (ASC, $$55) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- STABLE_SORT [$$58(ASC), $$56(ASC), $$55(ASC)] |PARTITIONED|
exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- join (eq($$52, $$55)) [cardinality: 2.1, op-cost: 2000000.0, total-cost: 1.1E7]
- -- HYBRID_HASH_JOIN [$$55][$$52] |PARTITIONED|
+ join (eq($$55, $$58)) [cardinality: 2.1, op-cost: 2000000.0, total-cost: 1.1E7]
+ -- HYBRID_HASH_JOIN [$$58][$$55] |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 7000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- join (eq($$53, $$55)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
- -- HYBRID_HASH_JOIN [$$55][$$53] |PARTITIONED|
+ join (eq($$56, $$58)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
+ -- HYBRID_HASH_JOIN [$$58][$$56] |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$55]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$58] |PARTITIONED|
+ project ([$$58]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$55, $$n] <- tpch.Nation [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$58, $$n] <- tpch.Nation [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
- project ([$$53]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$56] |PARTITIONED|
+ project ([$$56]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$53] <- [$$s.getField(3)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ assign [$$56] <- [$$s.getField(3)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ASSIGN |PARTITIONED|
project ([$$s]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$56, $$s] <- tpch.Supplier [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$59, $$s] <- tpch.Supplier [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- HASH_PARTITION_EXCHANGE [$$52] |PARTITIONED|
- project ([$$52]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$55] |PARTITIONED|
+ project ([$$55]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$52] <- [$$c.getField(3)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ assign [$$55] <- [$$c.getField(3)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ASSIGN |PARTITIONED|
project ([$$c]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$57, $$c] <- tpch.Customer [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$60, $$c] <- tpch.Customer [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan
index 3148a70..919037d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.06.plan
@@ -1,48 +1,48 @@
-distribute result [$$34] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+distribute result [$$36] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- DISTRIBUTE_RESULT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$34]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ project ([$$36]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$34] <- [{"o_orderkey": $$41, "l_orderkey": $$42, "l_suppkey": $$40}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ assign [$$36] <- [{"o_orderkey": $$43, "l_orderkey": $$44, "l_suppkey": $$42}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- ASSIGN |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
- -- SORT_MERGE_EXCHANGE [$$41(ASC), $$42(ASC), $$40(ASC) ] |PARTITIONED|
- order (ASC, $$41) (ASC, $$42) (ASC, $$40) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
- -- STABLE_SORT [$$41(ASC), $$42(ASC), $$40(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$43(ASC), $$44(ASC), $$42(ASC) ] |PARTITIONED|
+ order (ASC, $$43) (ASC, $$44) (ASC, $$42) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- STABLE_SORT [$$43(ASC), $$44(ASC), $$42(ASC)] |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$41, $$42, $$40]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ project ([$$43, $$44, $$42]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- join (and(eq($$41, $$42), eq($$47, $$40))) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
- -- HYBRID_HASH_JOIN [$$41, $$47][$$42, $$40] |PARTITIONED|
+ join (and(eq($$43, $$44), eq($$49, $$42))) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
+ -- HYBRID_HASH_JOIN [$$43, $$49][$$44, $$42] |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$47] <- [$$41] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$43, $$49] |PARTITIONED|
+ assign [$$49] <- [$$43] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ASSIGN |PARTITIONED|
- project ([$$41]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ project ([$$43]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$41, $$o] <- tpch.Orders [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$43, $$o] <- tpch.Orders [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- HASH_PARTITION_EXCHANGE [$$42] |PARTITIONED|
- project ([$$42, $$40]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$44, $$42] |PARTITIONED|
+ project ([$$44, $$42]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$40] <- [$$l.getField(2)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ assign [$$42] <- [$$l.getField(2)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ASSIGN |PARTITIONED|
- project ([$$42, $$l]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ project ([$$44, $$l]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$42, $$43, $$l] <- tpch.LineItem [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$44, $$45, $$l] <- tpch.LineItem [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan
index cf774fa..61de808 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.08.plan
@@ -1,48 +1,48 @@
-distribute result [$$36] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$38] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- DISTRIBUTE_RESULT |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$36]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ project ([$$38]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$36] <- [{"o_orderkey": $$41, "l_orderkey": $$42, "l_suppkey": $$45}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ assign [$$38] <- [{"o_orderkey": $$43, "l_orderkey": $$44, "l_suppkey": $$47}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ASSIGN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- SORT_MERGE_EXCHANGE [$$41(ASC), $$42(ASC), $$45(ASC) ] |PARTITIONED|
- order (ASC, $$41) (ASC, $$42) (ASC, $$45) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STABLE_SORT [$$41(ASC), $$42(ASC), $$45(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$43(ASC), $$44(ASC), $$47(ASC) ] |PARTITIONED|
+ order (ASC, $$43) (ASC, $$44) (ASC, $$47) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$43(ASC), $$44(ASC), $$47(ASC)] |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$41, $$42, $$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ project ([$$43, $$44, $$47]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- left outer join (and(eq($$41, $$42), eq($$54, $$45))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- HYBRID_HASH_JOIN [$$41, $$54][$$42, $$45] |PARTITIONED|
+ left outer join (and(eq($$43, $$44), eq($$56, $$47))) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HYBRID_HASH_JOIN [$$43, $$56][$$44, $$47] |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$54] <- [$$41] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$43, $$56] |PARTITIONED|
+ assign [$$56] <- [$$43] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ASSIGN |PARTITIONED|
- project ([$$41]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ project ([$$43]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$41, $$o] <- tpch.Orders [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ data-scan []<-[$$43, $$o] <- tpch.Orders [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- HASH_PARTITION_EXCHANGE [$$42] |PARTITIONED|
- project ([$$42, $$45]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$44, $$47] |PARTITIONED|
+ project ([$$44, $$47]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$45] <- [$$l.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ assign [$$47] <- [$$l.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ASSIGN |PARTITIONED|
- project ([$$42, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ project ([$$44, $$l]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$42, $$43, $$l] <- tpch.LineItem [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ data-scan []<-[$$44, $$45, $$l] <- tpch.LineItem [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan
index 2dd4acc..b11c296 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash-join-with-redundant-variable/hash-join-with-redundant-variable.14.plan
@@ -1,64 +1,64 @@
-distribute result [$$48] [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+distribute result [$$51] [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
-- DISTRIBUTE_RESULT |PARTITIONED|
exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$48]) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ project ([$$51]) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$48] <- [{"n_nationkey": $$56, "s_nationkey": $$53, "c_nationkey": $$52}] [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ assign [$$51] <- [{"n_nationkey": $$59, "s_nationkey": $$56, "c_nationkey": $$55}] [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
-- ASSIGN |PARTITIONED|
exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
- -- SORT_MERGE_EXCHANGE [$$56(ASC), $$53(ASC), $$52(ASC) ] |PARTITIONED|
- order (ASC, $$56) (ASC, $$53) (ASC, $$52) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
- -- STABLE_SORT [$$56(ASC), $$53(ASC), $$52(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$59(ASC), $$56(ASC), $$55(ASC) ] |PARTITIONED|
+ order (ASC, $$59) (ASC, $$56) (ASC, $$55) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ -- STABLE_SORT [$$59(ASC), $$56(ASC), $$55(ASC)] |PARTITIONED|
exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$56, $$53, $$52]) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
+ project ([$$59, $$56, $$55]) [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 2.1, op-cost: 0.0, total-cost: 1.1E7]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- join (and(eq($$52, $$56), eq($$53, $$63))) [cardinality: 2.1, op-cost: 2000000.0, total-cost: 1.1E7]
- -- HYBRID_HASH_JOIN [$$56, $$53][$$52, $$63] |PARTITIONED|
+ join (and(eq($$55, $$59), eq($$56, $$66))) [cardinality: 2.1, op-cost: 2000000.0, total-cost: 1.1E7]
+ -- HYBRID_HASH_JOIN [$$59, $$56][$$55, $$66] |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 7000000.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$53, $$56]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$59, $$56] |PARTITIONED|
+ project ([$$56, $$59]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 7000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$56, $$n] <- index-search("Nation", 0, "tpch", "Nation", true, true, 1, $$53, 1, $$53, true, true, true) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 6000000.0]
+ unnest-map [$$59, $$n] <- index-search("Nation", 0, "tpch", "Nation", true, true, 1, $$56, 1, $$56, true, true, true) [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 6000000.0]
-- BTREE_SEARCH |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$53) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STABLE_SORT [$$53(ASC)] |PARTITIONED|
+ order (ASC, $$56) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
- project ([$$53]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- HASH_PARTITION_EXCHANGE [$$56] |PARTITIONED|
+ project ([$$56]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$53] <- [$$s.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ assign [$$56] <- [$$s.getField(3)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ASSIGN |PARTITIONED|
project ([$$s]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$55, $$s] <- tpch.Supplier [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$58, $$s] <- tpch.Supplier [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- HASH_PARTITION_EXCHANGE [$$52] |PARTITIONED|
- assign [$$63] <- [$$52] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$55, $$66] |PARTITIONED|
+ assign [$$66] <- [$$55] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ASSIGN |PARTITIONED|
- project ([$$52]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ project ([$$55]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$52] <- [$$c.getField(3)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ assign [$$55] <- [$$c.getField(3)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ASSIGN |PARTITIONED|
project ([$$c]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$57, $$c] <- tpch.Customer [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$60, $$c] <- tpch.Customer [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.6.plan
index 0712c07..2c1c522 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.6.plan
@@ -1,18 +1,22 @@
-distribute result [$$15] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+distribute result [$$16] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit offset 98 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([$$15]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ project ([$$16]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$15] <- [{"id": $$17, "dblpid": $$paper.getField(1)}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ assign [$$16] <- [{"id": $$18, "dblpid": $$paper.getField(1)}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ASSIGN |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SORT_MERGE_EXCHANGE [$$17(ASC) ] |PARTITIONED|
- data-scan []<-[$$17, $$paper] <- test.DBLP1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- SORT_MERGE_EXCHANGE [$$18(ASC) ] |PARTITIONED|
+ order (ASC, $$18) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$18, $$paper] <- test.DBLP1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.plan
index cf6f26e..47503b1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.plan
@@ -4,31 +4,31 @@
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 5 offset 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- SORT_MERGE_EXCHANGE [$$17(ASC), $$18(ASC) ] |PARTITIONED|
- limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$17, $$18, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$23, $$24, 2, $$23, $$24, true, true, true) condition (and(lt($$c.getField(2), 150), lt($$c.getField(5), 10000))) limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ unnest-map [$$18, $$19, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$24, $$25, 2, $$24, $$25, true, true, true) condition (and(lt($$c.getField(2), 150), lt($$c.getField(5), 10000))) limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- BTREE_SEARCH |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$23) (ASC, $$24) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STABLE_SORT [$$23(ASC), $$24(ASC)] |PARTITIONED|
+ order (ASC, $$24) (ASC, $$25) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$24(ASC), $$25(ASC)] |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$23, $$24]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ project ([$$24, $$25]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$22, $$23, $$24] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$21, true, false, false) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ unnest-map [$$23, $$24, $$25] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$22, true, false, false) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- BTREE_SEARCH |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$21] <- [150] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ assign [$$22] <- [150] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ASSIGN |PARTITIONED|
empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan
index 818e376..9f25055 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.plan
@@ -1,40 +1,40 @@
-distribute result [$$19] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$20] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([$$19]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- SORT_MERGE_EXCHANGE [$$21(ASC), $$22(ASC) ] |PARTITIONED|
- project ([$$21, $$22, $$19]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$19] <- [{"shipdate": substring($$c.getField(10), 0, 4), "suppkey": gt($$20, 0)}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ASSIGN |PARTITIONED|
- limit 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_LIMIT |PARTITIONED|
- assign [$$20] <- [$$c.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ project ([$$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$20] <- [{"shipdate": substring($$c.getField(10), 0, 4), "suppkey": gt($$21, 0)}] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ limit 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ assign [$$21] <- [$$c.getField(2)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$21, $$22, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$26, $$27, 2, $$26, $$27, true, true, true) condition (lt($$c.getField(2), 150)) limit 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ unnest-map [$$22, $$23, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$27, $$28, 2, $$27, $$28, true, true, true) condition (lt($$c.getField(2), 150)) limit 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- BTREE_SEARCH |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$26) (ASC, $$27) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STABLE_SORT [$$26(ASC), $$27(ASC)] |PARTITIONED|
+ order (ASC, $$27) (ASC, $$28) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$27(ASC), $$28(ASC)] |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$26, $$27]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ project ([$$27, $$28]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$25, $$26, $$27] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$24, true, false, false) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ unnest-map [$$26, $$27, $$28] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$25, true, false, false) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- BTREE_SEARCH |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$24] <- [150] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ assign [$$25] <- [150] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ASSIGN |PARTITIONED|
empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.plan
index 968ced8..48f0a7f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.plan
@@ -4,31 +4,31 @@
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 5 offset 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- SORT_MERGE_EXCHANGE [$$14(ASC), $$15(ASC) ] |PARTITIONED|
- limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$14, $$15, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$19, $$20, 2, $$19, $$20, true, true, true) condition (lt($$c.getField(2), 150)) limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ unnest-map [$$15, $$16, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$20, $$21, 2, $$20, $$21, true, true, true) condition (lt($$c.getField(2), 150)) limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- BTREE_SEARCH |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$19) (ASC, $$20) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STABLE_SORT [$$19(ASC), $$20(ASC)] |PARTITIONED|
+ order (ASC, $$20) (ASC, $$21) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$20(ASC), $$21(ASC)] |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$19, $$20]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ project ([$$20, $$21]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$18, $$19, $$20] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$17, true, false, false) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ unnest-map [$$19, $$20, $$21] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$18, true, false, false) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- BTREE_SEARCH |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$17] <- [150] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ assign [$$18] <- [150] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ASSIGN |PARTITIONED|
empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.plan
index cacdbb8..b7957e4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.plan
@@ -4,31 +4,31 @@
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 5 offset 5 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- SORT_MERGE_EXCHANGE [$$16(ASC), $$17(ASC) ] |PARTITIONED|
- limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$c]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$16, $$17, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$21, $$22, 2, $$21, $$22, true, true, true) condition (lt($$c.getField(2), 150)) limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ unnest-map [$$17, $$18, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$22, $$23, 2, $$22, $$23, true, true, true) condition (lt($$c.getField(2), 150)) limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- BTREE_SEARCH |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$21) (ASC, $$22) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STABLE_SORT [$$21(ASC), $$22(ASC)] |PARTITIONED|
+ order (ASC, $$22) (ASC, $$23) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STABLE_SORT [$$22(ASC), $$23(ASC)] |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$21, $$22]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ project ([$$22, $$23]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$20, $$21, $$22] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$19, true, false, false) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ unnest-map [$$21, $$22, $$23] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$20, true, false, false) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- BTREE_SEARCH |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$19] <- [150] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ assign [$$20] <- [150] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ASSIGN |PARTITIONED|
empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.plan
index e8b44d5..2962b4d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.plan
@@ -1,10 +1,10 @@
-distribute result [$$202] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$210] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- DISTRIBUTE_RESULT |LOCAL|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |LOCAL|
- aggregate [$$202] <- [agg-sql-sum($$231)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ aggregate [$$210] <- [agg-sql-sum($$239)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- AGGREGATE |LOCAL|
- aggregate [$$231] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ aggregate [$$239] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- AGGREGATE |LOCAL|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
@@ -14,39 +14,35 @@
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- SORT_MERGE_EXCHANGE [$$134(ASC) ] |PARTITIONED|
- limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$134]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$134, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 10))) limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$142, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 10))) limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- SORT_MERGE_EXCHANGE [$$135(ASC) ] |PARTITIONED|
- limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$135]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$135, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), -10), le($$onek1.getField(2), -1))) limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$143, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), -10), le($$onek1.getField(2), -1))) limit 1000 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.plan
index 25c2869..420c5fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.plan
@@ -4,15 +4,15 @@
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 5 offset 5 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([$$paper]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SORT_MERGE_EXCHANGE [$$14(ASC) ] |PARTITIONED|
- limit 10 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ limit 10 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$paper]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$14, $$paper] <- test.DBLP1 condition (contains($$paper.getField(1), "kimL89")) limit 10 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$15, $$paper] <- test.DBLP1 condition (contains($$paper.getField(1), "kimL89")) limit 10 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.plan
index 5beb697..f4cb82f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.plan
@@ -1,4 +1,4 @@
-distribute result [$$35] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+distribute result [$$37] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
@@ -6,57 +6,57 @@
-- STREAM_LIMIT |UNPARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- project ([$$35]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ project ([$$37]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$35] <- [{"dblpid": $$36}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ assign [$$37] <- [{"dblpid": $$38}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- ASSIGN |PARTITIONED|
limit 2 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- STREAM_LIMIT |PARTITIONED|
- project ([$$36]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
+ project ([$$38]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 6000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- join (eq($$36, $$39)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
- -- HYBRID_HASH_JOIN [$$36][$$39] |PARTITIONED|
+ join (eq($$38, $$41)) [cardinality: 1000000.0, op-cost: 2000000.0, total-cost: 6000000.0]
+ -- HYBRID_HASH_JOIN [$$38][$$41] |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- HASH_PARTITION_EXCHANGE [$$36] |PARTITIONED|
- project ([$$36]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$38] |PARTITIONED|
+ project ([$$38]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$36] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ assign [$$38] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ASSIGN |PARTITIONED|
project ([$$d]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$37, $$d] <- test.DBLP1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$39, $$d] <- test.DBLP1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- HASH_PARTITION_EXCHANGE [$$39] |PARTITIONED|
- project ([$$39]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- HASH_PARTITION_EXCHANGE [$$41] |PARTITIONED|
+ project ([$$41]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |UNPARTITIONED|
- assign [$$39] <- [get-item($$28, 0).getField(0).getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ assign [$$41] <- [get-item($$30, 0).getField(0).getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ASSIGN |UNPARTITIONED|
- aggregate [$$28] <- [listify($$27)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ aggregate [$$30] <- [listify($$29)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- AGGREGATE |UNPARTITIONED|
limit 1 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([$$27]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
- -- SORT_MERGE_EXCHANGE [$$38(ASC) ] |PARTITIONED|
- project ([$$38, $$27]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$27] <- [{"d": $$d}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- limit 1 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ project ([$$29]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$29] <- [{"d": $$d}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ limit 1 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$d]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 2000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$38, $$d] <- test.DBLP1 condition (ends-with($$d.getField(1), "Blakeley95")) limit 1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$40, $$d] <- test.DBLP1 condition (ends-with($$d.getField(1), "Blakeley95")) limit 1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.plan
index 46283c5..8d337ea 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.plan
@@ -1,26 +1,26 @@
-distribute result [$$18] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+distribute result [$$19] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 1 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([$$18]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- project ([$$20, $$18]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$18] <- [{"$1": substring($$19, 0, 21)}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- limit 1 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$20, $$19]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$19] <- [$$DBLP1.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ project ([$$19]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19] <- [{"$1": substring($$20, 0, 21)}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ limit 1 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$20]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$20] <- [$$DBLP1.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$DBLP1]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$20, $$DBLP1] <- test.DBLP1 condition (gt($$DBLP1.getField(1), "series")) limit 1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$21, $$DBLP1] <- test.DBLP1 condition (gt($$DBLP1.getField(1), "series")) limit 1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.plan
index 06387be..2e28a2e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.plan
@@ -1,26 +1,26 @@
-distribute result [$$21] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+distribute result [$$22] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 2 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([$$21]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
- limit 2 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$24, $$21]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$21] <- [$$25.getField("lang")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
- project ([$$24, $$25]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$25] <- [$$t.getField("user")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ limit 2 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$22]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22] <- [$$26.getField("lang")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$26]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$26] <- [$$t.getField("user")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ASSIGN |PARTITIONED|
+ project ([$$t]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$24, $$t] <- test.TweetMessages condition (and(ge($$t.getField("user").getField("friends_count"), 0), le($$t.getField("user").getField("friends_count"), 150))) limit 2 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$25, $$t] <- test.TweetMessages condition (and(ge($$t.getField("user").getField("friends_count"), 0), le($$t.getField("user").getField("friends_count"), 150))) limit 2 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.plan
index 3e40aa9..5261022 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.plan
@@ -4,15 +4,15 @@
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 5 offset 5 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([$$paper]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SORT_MERGE_EXCHANGE [$$12(ASC) ] |PARTITIONED|
- limit 10 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ limit 10 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$paper]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$12, $$paper] <- test.DBLP1 limit 10 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$13, $$paper] <- test.DBLP1 limit 10 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.plan
index fa06729..e04ec86 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.plan
@@ -4,15 +4,15 @@
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 5 offset 5 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([$$paper]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SORT_MERGE_EXCHANGE [$$14(ASC) ] |PARTITIONED|
- limit 10 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ limit 10 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$paper]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$14, $$paper] <- test.DBLP1 limit 10 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ data-scan []<-[$$15, $$paper] <- test.DBLP1 limit 10 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
-- DATASOURCE_SCAN |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.plan
index 81a8266..f57113c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.plan
@@ -1,10 +1,10 @@
-distribute result [$$180] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+distribute result [$$188] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- DISTRIBUTE_RESULT |LOCAL|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |LOCAL|
- aggregate [$$180] <- [agg-sql-sum($$205)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ aggregate [$$188] <- [agg-sql-sum($$213)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- AGGREGATE |LOCAL|
- aggregate [$$205] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ aggregate [$$213] <- [agg-sql-count(1)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- AGGREGATE |LOCAL|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
@@ -14,39 +14,35 @@
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 100 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- SORT_MERGE_EXCHANGE [$$120(ASC) ] |PARTITIONED|
- limit 100 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$120]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$120, $$onek1] <- test.onek1 limit 100 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ limit 100 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$128, $$onek1] <- test.onek1 limit 100 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- SORT_MERGE_EXCHANGE [$$121(ASC) ] |PARTITIONED|
- limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$121]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- STREAM_PROJECT |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$121, $$onek1] <- test.onek1 limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$129, $$onek1] <- test.onek1 limit 10 [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.plan
index 0231545..4baf55d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.plan
@@ -1,37 +1,43 @@
-distribute result [$$75] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+distribute result [$$80] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 5 offset 5 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([$$75]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ project ([$$80]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$75] <- [get-item($$73, 0)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ assign [$$80] <- [get-item($$78, 0)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ASSIGN |PARTITIONED|
- project ([$$73]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ project ([$$78]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SORT_MERGE_EXCHANGE [$$77(ASC) ] |PARTITIONED|
- project ([$$73, $$77]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_PROJECT |PARTITIONED|
- subplan {
- aggregate [$$73] <- [listify($$72)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- AGGREGATE |LOCAL|
- assign [$$72] <- [object-remove(object-remove(object-remove($$t0, "title"), "authors"), "misc")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ASSIGN |LOCAL|
- unnest $$t0 <- scan-collection(to-array($$paper)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- UNNEST |LOCAL|
- nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- NESTED_TUPLE_SOURCE |LOCAL|
- } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SUBPLAN |PARTITIONED|
- limit 10 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_LIMIT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$82(ASC) ] |PARTITIONED|
+ limit 10 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (topK: 10) (ASC, $$82) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [topK: 10] [$$82(ASC)] |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$77, $$paper] <- test.DBLP1 limit 10 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ project ([$$78, $$82]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_PROJECT |PARTITIONED|
+ subplan {
+ aggregate [$$78] <- [listify($$77)] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- AGGREGATE |LOCAL|
+ assign [$$77] <- [object-remove(object-remove(object-remove($$t0, "title"), "authors"), "misc")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ASSIGN |LOCAL|
+ unnest $$t0 <- scan-collection(to-array($$paper)) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- UNNEST |LOCAL|
+ nested tuple source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ } [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- SUBPLAN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$82, $$paper] <- test.DBLP1 [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan
index 0c25386..24348ee 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan
@@ -1,22 +1,26 @@
-distribute result [$$28] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+distribute result [$$30] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- DISTRIBUTE_RESULT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$28]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ project ([$$30]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$28] <- [$$md.getField("name")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ assign [$$30] <- [$$md.getField("name")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ASSIGN |PARTITIONED|
project ([$$md]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
- select (neq(uuid(), uuid())) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$32(ASC) ] |PARTITIONED|
+ order (ASC, $$32) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$30, $$md] <- test.MyDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ select (neq(uuid(), uuid())) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$32, $$md] <- test.MyDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.6.plan
index dc9b1d7..87a0d71 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.6.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.6.plan
@@ -1,22 +1,26 @@
-distribute result [$$28] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+distribute result [$$30] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- DISTRIBUTE_RESULT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$28]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ project ([$$30]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$28] <- [$$md.getField("name")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ assign [$$30] <- [$$md.getField("name")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ASSIGN |PARTITIONED|
project ([$$md]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- STREAM_PROJECT |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
- select (neq(current-date(), date: { 1980-09-10 })) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
- -- STREAM_SELECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$32(ASC) ] |PARTITIONED|
+ order (ASC, $$32) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$30, $$md] <- test.MyDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ select (neq(current-date(), date: { 1980-09-10 })) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+ -- STREAM_SELECT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ data-scan []<-[$$32, $$md] <- test.MyDataset [cardinality: 1000000.0, op-cost: 1000000.0, total-cost: 1000000.0]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.plan
index c69e8a0..35995c9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.plan
@@ -6,27 +6,31 @@
-- STREAM_PROJECT |PARTITIONED|
exchange
-- SORT_MERGE_EXCHANGE [$$17(ASC), $$18(ASC) ] |PARTITIONED|
- select (eq($$l.getField(10), "1994-01-20"))
- -- STREAM_SELECT |PARTITIONED|
+ order (ASC, $$17) (ASC, $$18)
+ -- STABLE_SORT [$$17(ASC), $$18(ASC)] |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$17, $$18, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", false, false, 2, $$25, $$26, 2, $$25, $$26, true, true, true)
- -- BTREE_SEARCH |PARTITIONED|
+ select (eq($$l.getField(10), "1994-01-20"))
+ -- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$25) (ASC, $$26)
- -- STABLE_SORT [$$25(ASC), $$26(ASC)] |PARTITIONED|
+ unnest-map [$$17, $$18, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", false, false, 2, $$25, $$26, 2, $$25, $$26, true, true, true)
+ -- BTREE_SEARCH |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$25, $$26])
- -- STREAM_PROJECT |PARTITIONED|
+ order (ASC, $$25) (ASC, $$26)
+ -- STABLE_SORT [$$25(ASC), $$26(ASC)] |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$24, $$25, $$26] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", false, false, 1, $$22, 1, $$23, true, true, true)
- -- BTREE_SEARCH |PARTITIONED|
+ project ([$$25, $$26])
+ -- STREAM_PROJECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$22, $$23] <- ["1994-01-20", "1994-01-20"]
- -- ASSIGN |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ unnest-map [$$24, $$25, $$26] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", false, false, 1, $$22, 1, $$23, true, true, true)
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$22, $$23] <- ["1994-01-20", "1994-01-20"]
+ -- ASSIGN |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.plan
index c69e8a0..35995c9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.plan
@@ -6,27 +6,31 @@
-- STREAM_PROJECT |PARTITIONED|
exchange
-- SORT_MERGE_EXCHANGE [$$17(ASC), $$18(ASC) ] |PARTITIONED|
- select (eq($$l.getField(10), "1994-01-20"))
- -- STREAM_SELECT |PARTITIONED|
+ order (ASC, $$17) (ASC, $$18)
+ -- STABLE_SORT [$$17(ASC), $$18(ASC)] |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$17, $$18, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", false, false, 2, $$25, $$26, 2, $$25, $$26, true, true, true)
- -- BTREE_SEARCH |PARTITIONED|
+ select (eq($$l.getField(10), "1994-01-20"))
+ -- STREAM_SELECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- order (ASC, $$25) (ASC, $$26)
- -- STABLE_SORT [$$25(ASC), $$26(ASC)] |PARTITIONED|
+ unnest-map [$$17, $$18, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", false, false, 2, $$25, $$26, 2, $$25, $$26, true, true, true)
+ -- BTREE_SEARCH |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$25, $$26])
- -- STREAM_PROJECT |PARTITIONED|
+ order (ASC, $$25) (ASC, $$26)
+ -- STABLE_SORT [$$25(ASC), $$26(ASC)] |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- unnest-map [$$24, $$25, $$26] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", false, false, 1, $$22, 1, $$23, true, true, true)
- -- BTREE_SEARCH |PARTITIONED|
+ project ([$$25, $$26])
+ -- STREAM_PROJECT |PARTITIONED|
exchange
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- assign [$$22, $$23] <- ["1994-01-20", "1994-01-20"]
- -- ASSIGN |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ unnest-map [$$24, $$25, $$26] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", false, false, 1, $$22, 1, $$23, true, true, true)
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$22, $$23] <- ["1994-01-20", "1994-01-20"]
+ -- ASSIGN |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.08.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.08.plan
index fe48b86..0cc674e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.08.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.08.plan
@@ -1,26 +1,32 @@
-distribute result [$$85] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+distribute result [$$88] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- project ([$$85]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ project ([$$88]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- STREAM_PROJECT |UNPARTITIONED|
- assign [$$85] <- [{"id": $$88, "review": $$92}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ assign [$$88] <- [{"id": $$91, "review": $$95}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- ASSIGN |UNPARTITIONED|
- limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- STREAM_LIMIT |UNPARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- SORT_MERGE_EXCHANGE [$$88(ASC) ] |PARTITIONED|
- limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$88, $$92]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$92] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$88, $$d] <- test.DatasetWithKnownField condition (and(not(is-unknown(int64-default-null($$d.getField("year")))), not(is-unknown(int64-default-null($$d.getField("quarter")))), eq($$d.getField(1), "good"))) limit 3 [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ order (ASC, $$91) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STABLE_SORT [$$91(ASC)] |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$91, $$95]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$95] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$91, $$d] <- test.DatasetWithKnownField condition (and(not(is-unknown(int64-default-null($$d.getField("year")))), not(is-unknown(int64-default-null($$d.getField("quarter")))), eq($$d.getField(1), "good"))) limit 3 [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.10.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.10.plan
index f496dbf..77d3425 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.10.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.10.plan
@@ -1,26 +1,32 @@
-distribute result [$$61] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+distribute result [$$63] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([$$61]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ project ([$$63]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$61] <- [{"id": $$63, "review": $$67}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ assign [$$63] <- [{"id": $$65, "review": $$69}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- ASSIGN |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- SORT_MERGE_EXCHANGE [$$63(ASC) ] |PARTITIONED|
- project ([$$63, $$67]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$67] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- ASSIGN |PARTITIONED|
- limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- STREAM_LIMIT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$65(ASC) ] |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (topK: 3) (ASC, $$65) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STABLE_SORT [topK: 3] [$$65(ASC)] |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$63, $$d] <- test.DatasetWithKnownField limit 3 [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ project ([$$65, $$69]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$69] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$65, $$d] <- test.DatasetWithKnownField [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.12.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.12.plan
index 7d39583..a4b323f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.12.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.12.plan
@@ -1,26 +1,32 @@
-distribute result [$$91] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+distribute result [$$94] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
- project ([$$91]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ project ([$$94]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- STREAM_PROJECT |UNPARTITIONED|
- assign [$$91] <- [{"id": $$94, "review": $$98}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ assign [$$94] <- [{"id": $$97, "review": $$101}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- ASSIGN |UNPARTITIONED|
- limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- STREAM_LIMIT |UNPARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- SORT_MERGE_EXCHANGE [$$94(ASC) ] |PARTITIONED|
- limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- STREAM_LIMIT |PARTITIONED|
- project ([$$94, $$98]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$98] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- ASSIGN |PARTITIONED|
- exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$94, $$d] <- test.DatasetWithKnownField condition (and(not(is-unknown(int64-default-null($$d.getField("year")))), not(is-unknown(int64-default-null($$d.getField("quarter")))), eq($$d.getField(1), "good"))) limit 3 [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ order (ASC, $$97) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STABLE_SORT [$$97(ASC)] |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$97, $$101]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$101] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$97, $$d] <- test.DatasetWithKnownField condition (and(not(is-unknown(int64-default-null($$d.getField("year")))), not(is-unknown(int64-default-null($$d.getField("quarter")))), eq($$d.getField(1), "good"))) limit 3 [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.14.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.14.plan
index 5aa69a4..efe275f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.14.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-pushdown/view-pushdown.14.plan
@@ -1,26 +1,32 @@
-distribute result [$$67] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+distribute result [$$69] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- DISTRIBUTE_RESULT |UNPARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- STREAM_LIMIT |UNPARTITIONED|
- project ([$$67]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ project ([$$69]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- STREAM_PROJECT |PARTITIONED|
- assign [$$67] <- [{"id": $$69, "review": $$73}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ assign [$$69] <- [{"id": $$71, "review": $$75}] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- ASSIGN |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- SORT_MERGE_EXCHANGE [$$69(ASC) ] |PARTITIONED|
- project ([$$69, $$73]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$73] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- ASSIGN |PARTITIONED|
- limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
- -- STREAM_LIMIT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$71(ASC) ] |PARTITIONED|
+ limit 3 [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (topK: 3) (ASC, $$71) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STABLE_SORT [topK: 3] [$$71(ASC)] |PARTITIONED|
exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$69, $$d] <- test.DatasetWithKnownField limit 3 [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ project ([$$71, $$75]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$75] <- [$$d.getField(1)] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ASSIGN |PARTITIONED|
+ exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 2.1]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$71, $$d] <- test.DatasetWithKnownField [cardinality: 1000000.0, op-cost: 2.1, total-cost: 2.1]
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
index 324c660..81873af 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
@@ -47,7 +47,7 @@
}
@Override
- public IPhysicalPropertiesVector computePropertiesVector(List<LogicalVariable> scanVariables) {
+ public IPhysicalPropertiesVector computeRequiredProperties(List<LogicalVariable> scanVariables) {
IPhysicalPropertiesVector propsVector;
IPartitioningProperty pp;
List<ILocalStructuralProperty> propsLocal = new ArrayList<>();
@@ -73,6 +73,17 @@
return propsVector;
}
+ @Override
+ public IPhysicalPropertiesVector computeDeliveredProperties(List<LogicalVariable> scanVariables) {
+ if (ds.getDatasourceType() == DataSource.Type.INTERNAL_DATASET) {
+ IPartitioningProperty pp = new RandomPartitioningProperty(domain);
+ List<ILocalStructuralProperty> propsLocal = new ArrayList<>();
+ ds.computeLocalStructuralProperties(propsLocal, scanVariables);
+ return new StructuralPropertiesVector(pp, propsLocal);
+ }
+ return computeRequiredProperties(scanVariables);
+ }
+
private static List<OrderColumn> getOrderColumns(Set<LogicalVariable> pvars) {
List<OrderColumn> orderColumns = new ArrayList<>();
for (LogicalVariable pkVar : pvars) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
index 4dadf75..58377f4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
@@ -44,6 +44,7 @@
import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionFiltrationInfo;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
import org.apache.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -83,8 +84,17 @@
@Override
public IDataSourcePropertiesProvider getPropertiesProvider() {
// Unordered Random partitioning on all nodes
- return scanVariables -> new StructuralPropertiesVector(new RandomPartitioningProperty(domain),
- Collections.emptyList());
+ return new IDataSourcePropertiesProvider() {
+ @Override
+ public IPhysicalPropertiesVector computeRequiredProperties(List<LogicalVariable> scanVariables) {
+ return StructuralPropertiesVector.EMPTY_PROPERTIES_VECTOR;
+ }
+
+ @Override
+ public IPhysicalPropertiesVector computeDeliveredProperties(List<LogicalVariable> scanVariables) {
+ return new StructuralPropertiesVector(new RandomPartitioningProperty(domain), Collections.emptyList());
+ }
+ };
}
@Override
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index c55925c..2d3e0e4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -419,10 +419,11 @@
throws AlgebricksException {
DataSource source = findDataSource(dataSourceId);
Dataset dataset = ((DatasetDataSource) source).getDataset();
- Index secondaryIndex = getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexId);
- return (secondaryIndex != null)
- ? new DataSourceIndex(secondaryIndex, dataset.getDataverseName(), dataset.getDatasetName(), this)
- : null;
+ // index could be a primary index or secondary index
+ DataverseName dataverseName = dataset.getDataverseName();
+ String datasetName = dataset.getDatasetName();
+ Index index = getIndex(dataverseName, datasetName, indexId);
+ return index != null ? new DataSourceIndex(index, dataverseName, datasetName, this) : null;
}
public Index getIndex(DataverseName dataverseName, String datasetName, String indexName)
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IDataSourcePropertiesProvider.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IDataSourcePropertiesProvider.java
index e196dd1..5086e32 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IDataSourcePropertiesProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IDataSourcePropertiesProvider.java
@@ -24,5 +24,17 @@
import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
public interface IDataSourcePropertiesProvider {
- public IPhysicalPropertiesVector computePropertiesVector(List<LogicalVariable> scanVariables);
+ /**
+ *
+ * @param scanVariables
+ * @return
+ */
+ IPhysicalPropertiesVector computeRequiredProperties(List<LogicalVariable> scanVariables);
+
+ /**
+ *
+ * @param scanVariables
+ * @return
+ */
+ IPhysicalPropertiesVector computeDeliveredProperties(List<LogicalVariable> scanVariables);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
index bb0f08b..2543330 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
@@ -77,7 +77,7 @@
scanVariables.addAll(primaryKeys);
scanVariables.add(new LogicalVariable(-1));
IPhysicalPropertiesVector physicalProps =
- dataSource.getPropertiesProvider().computePropertiesVector(scanVariables);
+ dataSource.getPropertiesProvider().computeRequiredProperties(scanVariables);
StructuralPropertiesVector spv = new StructuralPropertiesVector(physicalProps.getPartitioningProperty(),
physicalProps.getLocalProperties());
return new PhysicalRequirements(new IPhysicalPropertiesVector[] { spv },
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
index 866334a..80843a4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
@@ -78,7 +78,7 @@
// partitioning properties
DataSourceScanOperator dssOp = (DataSourceScanOperator) op;
IDataSourcePropertiesProvider dspp = dataSource.getPropertiesProvider();
- deliveredProperties = dspp.computePropertiesVector(dssOp.getVariables());
+ deliveredProperties = dspp.computeDeliveredProperties(dssOp.getVariables());
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
index 7d70117..aee268b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
@@ -91,7 +91,7 @@
scanVariables.addAll(primaryKeys);
scanVariables.add(new LogicalVariable(-1));
IPhysicalPropertiesVector physicalProps =
- dataSourceIndex.getDataSource().getPropertiesProvider().computePropertiesVector(scanVariables);
+ dataSourceIndex.getDataSource().getPropertiesProvider().computeRequiredProperties(scanVariables);
List<ILocalStructuralProperty> localProperties = new ArrayList<>();
List<OrderColumn> orderColumns = new ArrayList<OrderColumn>();
// Data needs to be sorted based on the [token, number of token, PK]
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
index 3416163..0386275 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
@@ -109,7 +109,7 @@
scanVariables.add(new LogicalVariable(-1));
}
IPhysicalPropertiesVector r =
- dataSourceIndex.getDataSource().getPropertiesProvider().computePropertiesVector(scanVariables);
+ dataSourceIndex.getDataSource().getPropertiesProvider().computeRequiredProperties(scanVariables);
r.getLocalProperties().clear();
IPhysicalPropertiesVector[] requirements = new IPhysicalPropertiesVector[1];
requirements[0] = r;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
index 1d41291..7527fa6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
@@ -87,7 +87,7 @@
if (additionalNonFilteringFields != null) {
scanVariables.addAll(additionalNonFilteringFields);
}
- IPhysicalPropertiesVector r = dataSource.getPropertiesProvider().computePropertiesVector(scanVariables);
+ IPhysicalPropertiesVector r = dataSource.getPropertiesProvider().computeRequiredProperties(scanVariables);
r.getLocalProperties().clear();
IPhysicalPropertiesVector[] requirements = new IPhysicalPropertiesVector[1];
requirements[0] = r;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java
index 70e596a..1e97182 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java
@@ -84,7 +84,7 @@
List<LogicalVariable> scanVariables = new ArrayList<LogicalVariable>();
scanVariables.addAll(keys);
scanVariables.add(new LogicalVariable(-1));
- IPhysicalPropertiesVector r = dataSource.getPropertiesProvider().computePropertiesVector(scanVariables);
+ IPhysicalPropertiesVector r = dataSource.getPropertiesProvider().computeRequiredProperties(scanVariables);
IPhysicalPropertiesVector[] requirements = new IPhysicalPropertiesVector[1];
requirements[0] = r;
return new PhysicalRequirements(requirements, IPartitioningRequirementsCoordinator.NO_COORDINATION);